You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2009/08/02 00:30:49 UTC

svn commit: r799945 - in /incubator/cassandra/trunk: CHANGES.txt src/java/org/apache/cassandra/db/ColumnIndexer.java src/java/org/apache/cassandra/io/SequenceFile.java test/unit/org/apache/cassandra/db/TableTest.java

Author: jbellis
Date: Sat Aug  1 22:30:49 2009
New Revision: 799945

URL: http://svn.apache.org/viewvc?rev=799945&view=rev
Log:
r/m special case that no column names=all columns; add test for no-columns cases.
patch by jbellis; reviewed by Stu Hood for CASSANDRA-330

Modified:
    incubator/cassandra/trunk/CHANGES.txt
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java

Modified: incubator/cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/CHANGES.txt?rev=799945&r1=799944&r2=799945&view=diff
==============================================================================
--- incubator/cassandra/trunk/CHANGES.txt (original)
+++ incubator/cassandra/trunk/CHANGES.txt Sat Aug  1 22:30:49 2009
@@ -25,6 +25,9 @@
     - changed `block` boolean on insert methods to ConsistencyLevel enum,
       with options of NONE, ONE, QUORUM, and ALL.
     - added similar consistency_level parameter to read methods
+    - column-name-set slice with no names given now returns zero columns
+      instead of all of them.  ("all" can run your server out of memory.
+      use a range-based slice with a high max column count instead.)
  * Removed the web interface. Node information can now be obtained by 
    using the newly introduced nodeprobe utility.
  * More JMX stats

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java?rev=799945&r1=799944&r2=799945&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnIndexer.java Sat Aug  1 22:30:49 2009
@@ -123,11 +123,6 @@
             /* if we hit the column index size that we have to index after, go ahead and index it */
             if(position - sizeSummarized >= DatabaseDescriptor.getColumnIndexSize())
             {      
-                /*
-                 * ColumnSort applies only to columns. So in case of 
-                 * SuperColumn always use the name indexing scheme for
-                 * the SuperColumns. We will fix this later.
-                 */
                 IndexHelper.ColumnIndexInfo cIndexInfo = new IndexHelper.ColumnIndexInfo(column.name(), 0, 0, comparator);
                 cIndexInfo.position(position);
                 cIndexInfo.count(numColumns);                

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java?rev=799945&r1=799944&r2=799945&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SequenceFile.java Sat Aug  1 22:30:49 2009
@@ -328,93 +328,75 @@
             /* write the key into buffer */
             bufOut.writeUTF(key);
 
-            /* if we need to read the all the columns do not read the column indexes */
-            if (cNames == null || cNames.size() == 0)
+            /* Read the bloom filter summarizing the columns */
+            long preBfPos = file_.getFilePointer();
+            BloomFilter bf = defreezeBloomFilter();
+            long postBfPos = file_.getFilePointer();
+            dataSize -= (postBfPos - preBfPos);
+
+            List<IndexHelper.ColumnIndexInfo> columnIndexList = new ArrayList<IndexHelper.ColumnIndexInfo>();
+            /* read the column name indexes if present */
+            int totalBytesRead = handleColumnNameIndexes(columnFamilyName, columnIndexList);
+            dataSize -= totalBytesRead;
+
+            /* read the column family name */
+            String cfName = file_.readUTF();
+            dataSize -= (utfPrefix_ + cfName.length());
+
+            String cfType = file_.readUTF();
+            dataSize -= (utfPrefix_ + cfType.length());
+
+            String comparatorName = file_.readUTF();
+            dataSize -= (utfPrefix_ + comparatorName.length());
+
+            String subComparatorName = file_.readUTF();
+            dataSize -= (utfPrefix_ + subComparatorName.length());
+
+            /* read local deletion time */
+            int localDeletionTime = file_.readInt();
+            dataSize -=4;
+
+            /* read if this cf is marked for delete */
+            long markedForDeleteAt = file_.readLong();
+            dataSize -= 8;
+
+            /* read the total number of columns */
+            int totalNumCols = file_.readInt();
+            dataSize -= 4;
+
+            /* get the various column ranges we have to read */
+            List<IndexHelper.ColumnRange> columnRanges = IndexHelper.getMultiColumnRangesFromNameIndex(cNames, columnIndexList, dataSize, totalNumCols);
+
+            /* calculate the data size */
+            int numColsReturned = 0;
+            int dataSizeReturned = 0;
+            for (IndexHelper.ColumnRange columnRange : columnRanges)
             {
-                int bytesSkipped = IndexHelper.skipBloomFilterAndIndex(file_);
-                /*
-                       * read the correct number of bytes for the column family and
-                       * write data into buffer
-                      */
-                dataSize -= bytesSkipped;
-                /* write the data size */
-                bufOut.writeInt(dataSize);
-                /* write the data into buffer, except the boolean we have read */
-                bufOut.write(file_, dataSize);
+                numColsReturned += columnRange.count();
+                Coordinate coordinate = columnRange.coordinate();
+                dataSizeReturned += coordinate.end_ - coordinate.start_;
             }
-            else
+
+            // returned data size
+            bufOut.writeInt(dataSizeReturned + utfPrefix_ * 4 + cfName.length() + cfType.length() + comparatorName.length() + subComparatorName.length() + 4 + 4 + 8 + 4);
+            // echo back the CF data we read
+            bufOut.writeUTF(cfName);
+            bufOut.writeUTF(cfType);
+            bufOut.writeUTF(comparatorName);
+            bufOut.writeUTF(subComparatorName);
+            bufOut.writeInt(localDeletionTime);
+            bufOut.writeLong(markedForDeleteAt);
+            /* write number of columns */
+            bufOut.writeInt(numColsReturned);
+            int prevPosition = 0;
+            /* now write all the columns we are required to write */
+            for (IndexHelper.ColumnRange columnRange : columnRanges)
             {
-                /* Read the bloom filter summarizing the columns */
-                long preBfPos = file_.getFilePointer();
-                BloomFilter bf = defreezeBloomFilter();
-                long postBfPos = file_.getFilePointer();
-                dataSize -= (postBfPos - preBfPos);
-
-                List<IndexHelper.ColumnIndexInfo> columnIndexList = new ArrayList<IndexHelper.ColumnIndexInfo>();
-                /* read the column name indexes if present */
-                int totalBytesRead = handleColumnNameIndexes(columnFamilyName, columnIndexList);
-                dataSize -= totalBytesRead;
-
-                /* read the column family name */
-                String cfName = file_.readUTF();
-                dataSize -= (utfPrefix_ + cfName.length());
-
-                String cfType = file_.readUTF();
-                dataSize -= (utfPrefix_ + cfType.length());
-
-                String comparatorName = file_.readUTF();
-                dataSize -= (utfPrefix_ + comparatorName.length());
-
-                String subComparatorName = file_.readUTF();
-                dataSize -= (utfPrefix_ + subComparatorName.length());
-
-                /* read local deletion time */
-                int localDeletionTime = file_.readInt();
-                dataSize -=4;
-
-                /* read if this cf is marked for delete */
-                long markedForDeleteAt = file_.readLong();
-                dataSize -= 8;
-
-                /* read the total number of columns */
-                int totalNumCols = file_.readInt();
-                dataSize -= 4;
-
-                // TODO: this is name sorted - but eventually this should be sorted by the same criteria as the col index
-                /* get the various column ranges we have to read */
-                List<IndexHelper.ColumnRange> columnRanges = IndexHelper.getMultiColumnRangesFromNameIndex(cNames, columnIndexList, dataSize, totalNumCols);
-
-                /* calculate the data size */
-                int numColsReturned = 0;
-                int dataSizeReturned = 0;
-                for (IndexHelper.ColumnRange columnRange : columnRanges)
-                {
-                    numColsReturned += columnRange.count();
-                    Coordinate coordinate = columnRange.coordinate();
-                    dataSizeReturned += coordinate.end_ - coordinate.start_;
-                }
-
-                // returned data size
-                bufOut.writeInt(dataSizeReturned + utfPrefix_ * 4 + cfName.length() + cfType.length() + comparatorName.length() + subComparatorName.length() + 4 + 4 + 8 + 4);
-                // echo back the CF data we read
-                bufOut.writeUTF(cfName);
-                bufOut.writeUTF(cfType);
-                bufOut.writeUTF(comparatorName);
-                bufOut.writeUTF(subComparatorName);
-                bufOut.writeInt(localDeletionTime);
-                bufOut.writeLong(markedForDeleteAt);
-                /* write number of columns */
-                bufOut.writeInt(numColsReturned);
-                int prevPosition = 0;
-                /* now write all the columns we are required to write */
-                for (IndexHelper.ColumnRange columnRange : columnRanges)
-                {
-                    /* seek to the correct offset to the data */
-                    Coordinate coordinate = columnRange.coordinate();
-                    file_.skipBytes((int) (coordinate.start_ - prevPosition));
-                    bufOut.write(file_, (int) (coordinate.end_ - coordinate.start_));
-                    prevPosition = (int) coordinate.end_;
-                }
+                /* seek to the correct offset to the data */
+                Coordinate coordinate = columnRange.coordinate();
+                file_.skipBytes((int) (coordinate.start_ - prevPosition));
+                bufOut.write(file_, (int) (coordinate.end_ - coordinate.start_));
+                prevPosition = (int) coordinate.end_;
             }
         }
     }

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java?rev=799945&r1=799944&r2=799945&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java Sat Aug  1 22:30:49 2009
@@ -31,6 +31,7 @@
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryPath;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.io.SSTableReader;
 
@@ -58,7 +59,37 @@
         Row row = table.get("35300190:1");
         assertNotNull(row);
     }
-    
+
+    @Test
+    public void testGetRowNoColumns() throws Throwable
+    {
+        final Table table = Table.open("Keyspace2");
+        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Standard3");
+
+        Runner setup = new Runner()
+        {
+            public void run() throws Exception
+            {
+                RowMutation rm = makeSimpleRowMutation();
+                rm.apply();
+            }
+        };
+        Runner verify = new Runner()
+        {
+            public void run() throws Exception
+            {
+                ColumnFamily cf;
+
+                cf = cfStore.getColumnFamily(new NamesQueryFilter(TEST_KEY, new QueryPath("Standard3"), new TreeSet<byte[]>()));
+                assertColumns(cf);
+
+                cf = cfStore.getColumnFamily(new SliceQueryFilter(TEST_KEY, new QueryPath("Standard3"), ArrayUtils.EMPTY_BYTE_ARRAY, ArrayUtils.EMPTY_BYTE_ARRAY, true, 0));
+                assertColumns(cf);
+            }
+        };
+        reTest(setup, table.getColumnFamilyStore("Standard3"), verify);
+    }
+
     @Test
     public void testGetRowSingleColumn() throws Throwable
     {
@@ -88,7 +119,7 @@
         };
         reTest(setup, table.getColumnFamilyStore("Standard1"), verify);
     }
-            
+
     @Test
     public void testGetRowSliceByRange() throws Throwable
     {
@@ -119,7 +150,7 @@
 
     private RowMutation makeSimpleRowMutation()
     {
-        RowMutation rm = new RowMutation("Keyspace1",TEST_KEY);
+        RowMutation rm = new RowMutation("Keyspace1", TEST_KEY);
         ColumnFamily cf = ColumnFamily.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1","val1", 1L));
         cf.addColumn(column("col2","val2", 1L));