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/07/07 17:40:47 UTC

svn commit: r791872 [2/2] - in /incubator/cassandra/trunk: ./ interface/ interface/gen-java/org/apache/cassandra/service/ src/java/org/apache/cassandra/cli/ src/java/org/apache/cassandra/cql/common/ src/java/org/apache/cassandra/db/ src/java/org/apache...

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ReadCommand.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ReadCommand.java?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ReadCommand.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ReadCommand.java Tue Jul  7 15:40:46 2009
@@ -96,7 +96,6 @@
         CMD_SERIALIZER_MAP.put(ReadCommand.CMD_TYPE_GET_COLUMN, new ColumnReadCommandSerializer());
         CMD_SERIALIZER_MAP.put(ReadCommand.CMD_TYPE_GET_SLICE_BY_NAMES, new SliceByNamesReadCommandSerializer());
         CMD_SERIALIZER_MAP.put(ReadCommand.CMD_TYPE_GET_COLUMNS_SINCE, new ColumnsSinceReadCommandSerializer());
-        CMD_SERIALIZER_MAP.put(ReadCommand.CMD_TYPE_GET_SLICE_BY_RANGE, new SliceByRangeReadCommandSerializer());
         CMD_SERIALIZER_MAP.put(ReadCommand.CMD_TYPE_GET_SLICE_FROM, new SliceFromReadCommandSerializer());
     }
 

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SliceFromReadCommand.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SliceFromReadCommand.java?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SliceFromReadCommand.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/SliceFromReadCommand.java Tue Jul  7 15:40:46 2009
@@ -24,16 +24,19 @@
 public class SliceFromReadCommand extends ReadCommand
 {
     public final String columnFamilyColumn;
+    public final String start, finish;
     public final boolean isAscending;
-    public final int limit;
+    public final int offset;
     public final int count;
 
-    public SliceFromReadCommand(String table, String key, String columnFamilyColumn, boolean isAscending, int limit, int count)
+    public SliceFromReadCommand(String table, String key, String columnFamilyColumn, String start, String finish, boolean isAscending, int offset, int count)
     {
         super(table, key, CMD_TYPE_GET_SLICE_FROM);
         this.columnFamilyColumn = columnFamilyColumn;
+        this.start = start;
+        this.finish = finish;
         this.isAscending = isAscending;
-        this.limit = limit;
+        this.offset = offset;
         this.count = count;
     }
 
@@ -46,7 +49,7 @@
     @Override
     public ReadCommand copy()
     {
-        ReadCommand readCommand = new SliceFromReadCommand(table, key, columnFamilyColumn, isAscending, limit, count);
+        ReadCommand readCommand = new SliceFromReadCommand(table, key, columnFamilyColumn, start, finish, isAscending, offset, count);
         readCommand.setDigestQuery(isDigestQuery());
         return readCommand;
     }
@@ -54,7 +57,7 @@
     @Override
     public Row getRow(Table table) throws IOException
     {
-        return table.getSliceFrom(key, columnFamilyColumn, isAscending, limit, count);
+        return table.getRow(key, columnFamilyColumn, start, finish, isAscending, offset, count);
     }
 
     @Override
@@ -64,9 +67,11 @@
                "table='" + table + '\'' +
                ", key='" + key + '\'' +
                ", columnFamily='" + columnFamilyColumn + '\'' +
-               ", isAscending='" + isAscending + '\'' +
-               ", limit='" + limit + '\'' +
-               ", count='" + count + '\'' +
+               ", start='" + start + '\'' +
+               ", finish='" + finish + '\'' +
+               ", isAscending=" + isAscending +
+               ", offset=" + offset +
+               ", count=" + count +
                ')';
     }
 }
@@ -81,8 +86,10 @@
         dos.writeUTF(realRM.table);
         dos.writeUTF(realRM.key);
         dos.writeUTF(realRM.columnFamilyColumn);
+        dos.writeUTF(realRM.start);
+        dos.writeUTF(realRM.finish);
         dos.writeBoolean(realRM.isAscending);
-        dos.writeInt(realRM.limit);
+        dos.writeInt(realRM.offset);
         dos.writeInt(realRM.count);
     }
 
@@ -90,14 +97,7 @@
     public ReadCommand deserialize(DataInputStream dis) throws IOException
     {
         boolean isDigest = dis.readBoolean();
-        String table = dis.readUTF();
-        String key = dis.readUTF();
-        String columnFamily = dis.readUTF();
-        boolean isAscending = dis.readBoolean();
-        int limit = dis.readInt();
-        int count = dis.readInt();
-
-        SliceFromReadCommand rm = new SliceFromReadCommand(table, key, columnFamily, isAscending, limit, count);
+        SliceFromReadCommand rm = new SliceFromReadCommand(dis.readUTF(), dis.readUTF(), dis.readUTF(), dis.readUTF(), dis.readUTF(), dis.readBoolean(), dis.readInt(), dis.readInt());
         rm.setDigestQuery(isDigest);
         return rm;
     }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Tue Jul  7 15:40:46 2009
@@ -543,25 +543,6 @@
         	row.addColumnFamily(columnFamily);
         return row;
     }
-      
-    public Row getRow(String key, String cf, String startColumn, String endColumn, int count) throws IOException
-    {
-        Row row = new Row(table_, key);
-        String[] values = RowMutation.getColumnAndColumnFamily(cf);
-        ColumnFamilyStore cfStore = columnFamilyStores_.get(values[0]);
-        long start1 = System.currentTimeMillis();
-        assert cfStore != null : "Column family " + cf + " has not been defined";
-        ColumnFamily columnFamily = cfStore.getColumnFamily(key, cf, new IdentityFilter());
-        if ( columnFamily != null )
-        {
-            ColumnFamily filteredCf =  new RangeFilter(startColumn, endColumn, count).filter(cf, columnFamily);
-            row.addColumnFamily(filteredCf);
-        }
-        long timeTaken = System.currentTimeMillis() - start1;
-        dbAnalyticsSource_.updateReadStatistics(timeTaken);
-        return row;
-    }
-
     
     public Row getRow(String key, String cf, long sinceTimeStamp) throws IOException
     {
@@ -604,17 +585,14 @@
     /**
      * Selects a list of columns in a column family from a given column for the specified key.
     */
-    public Row getSliceFrom(String key, String cf, boolean isAscending, int limit, int count) throws IOException
+    public Row getRow(String key, String cfName, String start, String finish, boolean isAscending, int offset, int count) throws IOException
     {
         Row row = new Row(table_, key);
-        String[] values = cf.split(":", -1);
-        String cfName = values[0];
-        String startWith = values[1];
         ColumnFamilyStore cfStore = columnFamilyStores_.get(cfName);
         long start1 = System.currentTimeMillis();
         try
         {
-            ColumnFamily columnFamily = cfStore.getSliceFrom(key, cfName, startWith, isAscending, limit, count);
+            ColumnFamily columnFamily = cfStore.getSliceFrom(key, cfName, start, finish, isAscending, offset, count);
             if (columnFamily != null)
                 row.addColumnFamily(columnFamily);
             long timeTaken = System.currentTimeMillis() - start1;

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java Tue Jul  7 15:40:46 2009
@@ -167,18 +167,18 @@
         }
     }
 
-    public List<column_t> get_slice(String tablename, String key, String columnParent, boolean isAscending, int offset, int count) throws InvalidRequestException
+    public List<column_t> get_slice(String tablename, String key, String columnParent, String start, String finish, boolean isAscending, int offset, int count) throws InvalidRequestException, NotFoundException, TException
     {
         logger.debug("get_slice_from");
-        String[] values = columnParent.split(":", -1); // allow empty column specifier
-        if (values.length != 2 || !DatabaseDescriptor.getColumnFamilyType(tablename, values[0]).equals("Standard"))
+        String[] values = RowMutation.getColumnAndColumnFamily(columnParent);
+        if (values.length != 1 || !DatabaseDescriptor.getColumnFamilyType(tablename, values[0]).equals("Standard"))
             throw new InvalidRequestException("get_slice_from requires a standard CF name and a starting column name");
         if (count <= 0)
             throw new InvalidRequestException("get_slice_from requires positive count");
         if ("Name".compareTo(DatabaseDescriptor.getCFMetaData(tablename, values[0]).indexProperty_) != 0)
             throw new InvalidRequestException("get_slice_from requires CF indexed by name");
 
-        ColumnFamily cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnParent, isAscending, offset, count));
+        ColumnFamily cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnParent, start, finish, isAscending, offset, count));
         if (cfamily == null)
         {
             return EMPTY_COLUMNS;
@@ -232,7 +232,7 @@
         ColumnFamily cfamily;
         if (DatabaseDescriptor.isNameSortingEnabled(tablename, values[0]))
         {
-            cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnParent + ":", true, 0, Integer.MAX_VALUE));
+            cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnParent, "", "", true, 0, Integer.MAX_VALUE));
         }
         else
         {
@@ -341,16 +341,16 @@
         return thriftSuperColumns;
     }
 
-    public List<superColumn_t> get_slice_super(String tablename, String key, String columnFamily, boolean isAscending, int offset, int count) throws InvalidRequestException
+    public List<superColumn_t> get_slice_super(String tablename, String key, String columnFamily, String start, String finish, boolean isAscending, int offset, int count) throws InvalidRequestException, TException
     {
         logger.debug("get_slice_super");
-        String[] values = columnFamily.split(":", -1);
-        if (values.length != 2 || !DatabaseDescriptor.getColumnFamilyType(tablename, values[0]).equals("Super"))
-            throw new InvalidRequestException("get_slice_super requires a super CF name and a starting column name");
+        String[] values = RowMutation.getColumnAndColumnFamily(columnFamily);
+        if (values.length != 1 || !DatabaseDescriptor.getColumnFamilyType(tablename, values[0]).equals("Super"))
+            throw new InvalidRequestException("get_slice_super requires a super CF name");
         if (count <= 0)
             throw new InvalidRequestException("get_slice_super requires positive count");
 
-        ColumnFamily cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnFamily, isAscending, offset, count));
+        ColumnFamily cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnFamily, start, finish, isAscending, offset, count));
         if (cfamily == null)
         {
             return EMPTY_SUPERCOLUMNS;
@@ -516,31 +516,5 @@
         return StorageProxy.getKeyRange(new RangeCommand(tablename, columnFamilies, startWith, stopAt, maxResults));
     }
 
-	public List<column_t> get_slice_by_name_range(String tablename, String key, String columnParent, String start, String finish, int count)
-    throws InvalidRequestException, NotFoundException, TException
-    {
-		logger.debug("get_slice_by_name_range");
-        String[] values = ThriftValidation.validateColumnParent(tablename, columnParent);
-
-        ColumnFamily cfamily = readColumnFamily(new SliceByRangeReadCommand(tablename, key, columnParent, start, finish, count));
-        if (cfamily == null)
-        {
-            return EMPTY_COLUMNS;
-        }
-        if (DatabaseDescriptor.getColumnFamilyType(tablename, values[0]).equals("Standard"))
-        {
-            return thriftifyColumns(cfamily.getAllColumns());
-        }
-        else
-        {
-            IColumn superColumn = cfamily.getColumn(values[1]);
-            if (superColumn == null)
-            {
-                return EMPTY_COLUMNS;
-            }
-            return thriftifyColumns(superColumn.getSubColumns());
-        }
-	}
-
     // main method moved to CassandraDaemon
 }

Modified: incubator/cassandra/trunk/test/system/test_server.py
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/test_server.py?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/system/test_server.py (original)
+++ incubator/cassandra/trunk/test/system/test_server.py Tue Jul  7 15:40:46 2009
@@ -43,13 +43,13 @@
 
 def _verify_batch():
     _verify_simple()
-    L = client.get_slice('Table1', 'key1', 'Standard2:', True, 0, 1000)
+    L = client.get_slice('Table1', 'key1', 'Standard2', '', '', True, 0, 1000)
     assert L == _SIMPLE_COLUMNS, L
 
 def _verify_simple():
     assert client.get_column('Table1', 'key1', 'Standard1:c1') == \
         column_t(columnName='c1', value='value1', timestamp=0)
-    L = client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000)
+    L = client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000)
     assert L == _SIMPLE_COLUMNS, L
 
 def _insert_super():
@@ -65,22 +65,22 @@
     time.sleep(0.1)
 
 def _verify_range():
-    result = client.get_slice_by_name_range('Table1','key1', 'Standard1', 'c1', 'c2' , -1)
+    result = client.get_slice('Table1','key1', 'Standard1', 'c1', 'c2', True, 0, 1000)
     assert len(result) == 2
     assert result[0].columnName == 'c1'
     assert result[1].columnName == 'c2'
 
-    result = client.get_slice_by_name_range('Table1','key1', 'Standard1', 'a', 'z' , -1)
+    result = client.get_slice('Table1','key1', 'Standard1', 'a', 'z' , True, 0, 1000)
     assert len(result) == 3, result
     
-    result = client.get_slice_by_name_range('Table1','key1', 'Standard1', 'a', 'z' , 2)
+    result = client.get_slice('Table1','key1', 'Standard1', 'a', 'z' , True, 0, 2)
     assert len(result) == 2, result
 
 	 	
 def _verify_super(supercolumn='Super1'):
     assert client.get_column('Table1', 'key1', supercolumn + ':sc1:c4') == \
         column_t(columnName='c4', value='value4', timestamp=0)
-    slice = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+    slice = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
     assert slice == _SUPER_COLUMNS, slice
 
 def _expect_exception(fn, type_):
@@ -101,10 +101,10 @@
         _verify_simple()
 
     def test_empty_slice(self):
-        assert client.get_slice('Table1', 'key1', 'Standard2:', True, 0, 1000) == []
+        assert client.get_slice('Table1', 'key1', 'Standard2', '', '', True, 0, 1000) == []
 
     def test_empty_slice_super(self):
-        assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000) == []
+        assert client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000) == []
 
     def test_missing_super(self):
         _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc1:c1'))
@@ -160,24 +160,24 @@
         _expect_missing(lambda: client.get_column('Table1', 'key1', 'Standard1:c1'))
         assert client.get_column('Table1', 'key1', 'Standard1:c2') == \
             column_t(columnName='c2', value='value2', timestamp=0)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == \
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == \
             [column_t(columnName='c2', value='value2', timestamp=0)]
 
         # New insert, make sure it shows up post-remove:
         client.insert('Table1', 'key1', 'Standard1:c3', 'value3', 0, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == \
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == \
             [column_t(columnName='c2', value='value2', timestamp=0), 
              column_t(columnName='c3', value='value3', timestamp=0)]
 
         # Test resurrection.  First, re-insert the value w/ older timestamp, 
-        # and make sure it stays removed:
+        # and make sure it stays removed
         client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 0, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == \
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == \
             [column_t(columnName='c2', value='value2', timestamp=0), 
              column_t(columnName='c3', value='value3', timestamp=0)]
         # Next, w/ a newer timestamp; it should come back:
         client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 2, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == \
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == \
             [column_t(columnName='c1', value='value1', timestamp=2),
              column_t(columnName='c2', value='value2', timestamp=0), 
              column_t(columnName='c3', value='value3', timestamp=0)]
@@ -189,16 +189,16 @@
 
         # Remove the key1:Standard1 cf:
         client.remove('Table1', 'key1', 'Standard1', 3, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == []
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == []
         _verify_super()
 
         # Test resurrection.  First, re-insert a value w/ older timestamp, 
         # and make sure it stays removed:
         client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 0, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == []
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == []
         # Next, w/ a newer timestamp; it should come back:
         client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 4, True)
-        assert client.get_slice('Table1', 'key1', 'Standard1:', True, 0, 1000) == \
+        assert client.get_slice('Table1', 'key1', 'Standard1', '', '', True, 0, 1000) == \
             [column_t(columnName='c1', value='value1', timestamp=4)]
 
 
@@ -209,7 +209,7 @@
         # Make sure remove clears out what it's supposed to, and _only_ that:
         client.remove('Table1', 'key1', 'Super1:sc2:c5', 5, True)
         _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc2:c5'))
-        assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000) == \
+        assert client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000) == \
             [superColumn_t(name='sc1', 
                            columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
              superColumn_t(name='sc2', 
@@ -224,17 +224,17 @@
                              columns=[column_t(columnName='c6', value='value6', timestamp=0),
                                       column_t(columnName='c7', value='value7', timestamp=0)])]
 
-        assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000) == scs
+        assert client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000) == scs
 
         # Test resurrection.  First, re-insert the value w/ older timestamp, 
         # and make sure it stays removed:
         client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 0, True)
-        actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
         assert actual == scs, actual
 
         # Next, w/ a newer timestamp; it should come back
         client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 6, True)
-        actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
         assert actual == \
             [superColumn_t(name='sc1', 
                            columns=[column_t(columnName='c4', value='value4', timestamp=0)]), 
@@ -254,19 +254,19 @@
         assert actual == [], actual
         scs = [superColumn_t(name='sc1', 
                              columns=[column_t(columnName='c4', value='value4', timestamp=0)])]
-        actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
         assert actual == scs, actual
         _verify_simple()
 
         # Test resurrection.  First, re-insert the value w/ older timestamp, 
         # and make sure it stays removed:
         client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 0, True)
-        actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
         assert actual == scs, actual
 
         # Next, w/ a newer timestamp; it should come back
         client.insert('Table1', 'key1', 'Super1:sc2:c5', 'value5', 6, True)
-        actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 0, 1000)
+        actual = client.get_slice_super('Table1', 'key1', 'Super1', '', '', True, 0, 1000)
         assert actual == \
             [superColumn_t(name='sc1', 
                            columns=[column_t(columnName='c4', value='value4', timestamp=0)]),
@@ -311,14 +311,9 @@
         L = client.get_key_range('Table1', [], '1', '', 10)
         assert L == ['1', '10', '11', '12', '13', '14', '15', '16', '17', '18'], L
 
-    def test_get_slice_by_name_range(self):
+    def test_get_slice_range(self):
 	_insert_range()
 	_verify_range()
-
-        _insert_super()
-        result = client.get_slice_by_name_range('Table1','key1', 'Super1:sc1', 'a', 'z', -1)
-        assert len(result) == 1, result
-        assert result[0].columnName == 'c4'
         
     def test_get_slice_by_names(self):
         _insert_range()

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java?rev=791872&r1=791871&r2=791872&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ReadMessageTest.java Tue Jul  7 15:40:46 2009
@@ -57,11 +57,11 @@
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Table1", "row1", "foo", true, 0, 2);
+        rm = new SliceFromReadCommand("Table1", "row1", "foo", "", "", true, 0, 2);
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
         
-        rm = new SliceByRangeReadCommand("Table1", "row1", "foo", "a", "z", 5);
+        rm = new SliceFromReadCommand("Table1", "row1", "foo", "a", "z", true, 0, 5);
         rm2 = serializeAndDeserializeReadMessage(rm);
         assertEquals(rm2.toString(), rm.toString());
     }

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=791872&r1=791871&r2=791872&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 Tue Jul  7 15:40:46 2009
@@ -95,18 +95,17 @@
         rm.add(cf);
         rm.apply();
         
-        Row result = table.getRow(key, "Standard1", "b", "c",-1);
+        Row result = table.getRow(key, "Standard1", "b", "c", true, 0, 100);
         assertEquals(2, result.getColumnFamily("Standard1").getColumnCount());
         
-        result = table.getRow(key, "Standard1", "b", "b", 50);
+        result = table.getRow(key, "Standard1", "b", "b", true, 0, 100);
         assertEquals(1, result.getColumnFamily("Standard1").getColumnCount());
         
-        result = table.getRow(key, "Standard1", "b", "c",1);
+        result = table.getRow(key, "Standard1", "b", "c", true, 0, 1);
         assertEquals(1, result.getColumnFamily("Standard1").getColumnCount());
         
-        result = table.getRow(key, "Standard1", "c", "b",1);
-        assertEquals(0, result.getColumnFamily("Standard1").getColumnCount());
-        
+        result = table.getRow(key, "Standard1", "c", "b", true, 0, 1);
+        assertNull(result.getColumnFamily("Standard1"));
     }
 
     private RowMutation makeSimpleRowMutation()
@@ -146,12 +145,12 @@
         ColumnFamily cf;
 
         // key before the rows that exists
-        result = table.getSliceFrom("a", "Standard2:", true, 0, 1);
+        result = table.getRow("a", "Standard2", "", "", true, 0, 1);
         cf = result.getColumnFamily("Standard2");
         assertColumns(cf);
 
         // key after the rows that exist
-        result = table.getSliceFrom("z", "Standard2:", true, 0, 1);
+        result = table.getRow("z", "Standard2", "", "", true, 0, 1);
         cf = result.getColumnFamily("Standard2");
         assertColumns(cf);
     }
@@ -190,27 +189,27 @@
                 Row result;
                 ColumnFamily cf;
 
-                result = table.getSliceFrom(ROW, "Standard1:col5", true, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col5", "", true, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf, "col5", "col7");
 
-                result = table.getSliceFrom(ROW, "Standard1:col4", true, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col4", "", true, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf, "col5", "col7");
 
-                result = table.getSliceFrom(ROW, "Standard1:col5", false, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col5", "", false, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf, "col3", "col4", "col5");
 
-                result = table.getSliceFrom(ROW, "Standard1:col6", false, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col6", "", false, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf, "col3", "col4", "col5");
 
-                result = table.getSliceFrom(ROW, "Standard1:col95", true, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col95", "", true, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf);
 
-                result = table.getSliceFrom(ROW, "Standard1:col0", false, 0, 2);
+                result = table.getRow(ROW, "Standard1", "col0", "", false, 0, 2);
                 cf = result.getColumnFamily("Standard1");
                 assertColumns(cf);
             }
@@ -258,7 +257,7 @@
                 Row result;
                 ColumnFamily cfres;
 
-                result = table.getSliceFrom(ROW, "Standard1:col2", true, 0, 3);
+                result = table.getRow(ROW, "Standard1", "col2", "", true, 0, 3);
                 cfres = result.getColumnFamily("Standard1");
                 assertColumns(cfres, "col2", "col3", "col4");
                 assertEquals(new String(cfres.getColumn("col2").value()), "valx");
@@ -286,42 +285,42 @@
 
         Row result;
         ColumnFamily cfres;
-        result = table.getSliceFrom(ROW, "Standard1:col1000", true, 0, 3);
+        result = table.getRow(ROW, "Standard1", "col1000", "", true, 0, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1000", "col1001", "col1002");
         assertEquals(new String(cfres.getColumn("col1000").value()), "vvvvvvvvvvvvvvvv1000");
         assertEquals(new String(cfres.getColumn("col1001").value()), "vvvvvvvvvvvvvvvv1001");
         assertEquals(new String(cfres.getColumn("col1002").value()), "vvvvvvvvvvvvvvvv1002");
 
-        result = table.getSliceFrom(ROW, "Standard1:col1195", true, 0, 3);
+        result = table.getRow(ROW, "Standard1", "col1195", "", true, 0, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1195", "col1196", "col1197");
         assertEquals(new String(cfres.getColumn("col1195").value()), "vvvvvvvvvvvvvvvv1195");
         assertEquals(new String(cfres.getColumn("col1196").value()), "vvvvvvvvvvvvvvvv1196");
         assertEquals(new String(cfres.getColumn("col1197").value()), "vvvvvvvvvvvvvvvv1197");
 
-        result = table.getSliceFrom(ROW, "Standard1:col1195", true, 10, 3);
+        result = table.getRow(ROW, "Standard1", "col1195", "", true, 10, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1205", "col1206", "col1207");
         assertEquals(new String(cfres.getColumn("col1205").value()), "vvvvvvvvvvvvvvvv1205");
         assertEquals(new String(cfres.getColumn("col1206").value()), "vvvvvvvvvvvvvvvv1206");
         assertEquals(new String(cfres.getColumn("col1207").value()), "vvvvvvvvvvvvvvvv1207");
 
-        result = table.getSliceFrom(ROW, "Standard1:col1196", false, 0, 3);
+        result = table.getRow(ROW, "Standard1", "col1196", "", false, 0, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1194", "col1195", "col1196");
         assertEquals(new String(cfres.getColumn("col1194").value()), "vvvvvvvvvvvvvvvv1194");
         assertEquals(new String(cfres.getColumn("col1195").value()), "vvvvvvvvvvvvvvvv1195");
         assertEquals(new String(cfres.getColumn("col1196").value()), "vvvvvvvvvvvvvvvv1196");
 
-        result = table.getSliceFrom(ROW, "Standard1:col1196", false, 10, 3);
+        result = table.getRow(ROW, "Standard1", "col1196", "", false, 10, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1184", "col1185", "col1186");
         assertEquals(new String(cfres.getColumn("col1184").value()), "vvvvvvvvvvvvvvvv1184");
         assertEquals(new String(cfres.getColumn("col1185").value()), "vvvvvvvvvvvvvvvv1185");
         assertEquals(new String(cfres.getColumn("col1186").value()), "vvvvvvvvvvvvvvvv1186");
 
-        result = table.getSliceFrom(ROW, "Standard1:col1990", true, 0, 3);
+        result = table.getRow(ROW, "Standard1", "col1990", "", true, 0, 3);
         cfres = result.getColumnFamily("Standard1");
         assertColumns(cfres, "col1990", "col1991", "col1992");
         assertEquals(new String(cfres.getColumn("col1990").value()), "vvvvvvvvvvvvvvvv1990");
@@ -356,7 +355,7 @@
                 Row result;
                 ColumnFamily cfres;
 
-                result = table.getSliceFrom(ROW, "Super1:", true, 0, 10);
+                result = table.getRow(ROW, "Super1", "", "", true, 0, 10);
                 cfres = result.getColumnFamily("Super1");
                 assertColumns(cfres, "sc1");
                 assertEquals(new String(cfres.getColumn("sc1").getSubColumn("col1").value()), "val1");