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=788516&r1=788515&r2=788516&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 Thu Jun 25 21:26:33 2009 @@ -539,37 +539,7 @@ row.addColumnFamily(columnFamily); return row; } - - /** - * Selects only the specified column family for the specified key. - */ - public Row getRow(String key, String cf, int start, 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 = null; - if ((count <=0 || count == Integer.MAX_VALUE) && start <= 0) //Don't need to filter - { - filteredCf = columnFamily; - } - else - { - filteredCf = new CountFilter(count, start).filter(cf, columnFamily); - } - row.addColumnFamily(filteredCf); - } - long timeTaken = System.currentTimeMillis() - start1; - dbAnalyticsSource_.updateReadStatistics(timeTaken); - return row; - } - + public Row getRow(String key, String cf, String startColumn, String endColumn, int count) throws IOException { Row row = new Row(table_, key); @@ -633,9 +603,9 @@ public Row getSliceFrom(String key, String cf, boolean isAscending, int count) throws IOException { Row row = new Row(table_, key); - String[] values = RowMutation.getColumnAndColumnFamily(cf); + String[] values = cf.split(":", -1); String cfName = values[0]; - String startWith = values.length > 1 ? values[1] : ""; + String startWith = values[1]; ColumnFamilyStore cfStore = columnFamilyStores_.get(cfName); long start1 = System.currentTimeMillis(); try
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=788516&r1=788515&r2=788516&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 Thu Jun 25 21:26:33 2009 @@ -175,36 +175,12 @@ } return thriftifyColumns(cfamily.getAllColumns()); } - - public List<column_t> get_slice(String tablename, String key, String columnParent, int start, int count) throws InvalidRequestException - { - logger.debug("get_slice"); - String[] values = RowMutation.getColumnAndColumnFamily(columnParent); - ColumnFamily cfamily = readColumnFamily(new SliceReadCommand(tablename, key, columnParent, start, count)); - if (cfamily == null) - { - return EMPTY_COLUMNS; - } - Collection<IColumn> columns = null; - if( values.length > 1 ) - { - // this is the super column case - IColumn column = cfamily.getColumn(values[1]); - if(column != null) - columns = column.getSubColumns(); - } - else - { - columns = cfamily.getAllColumns(); - } - return thriftifyColumns(columns); - } - public List<column_t> get_slice_from(String tablename, String key, String columnParent, boolean isAscending, int count) throws InvalidRequestException + public List<column_t> get_slice(String tablename, String key, String columnParent, boolean isAscending, int count) throws InvalidRequestException { logger.debug("get_slice_from"); - String[] values = RowMutation.getColumnAndColumnFamily(columnParent); - if (values.length != 2 || DatabaseDescriptor.getColumnFamilyType(tablename, values[0]) != "Standard") + String[] values = columnParent.split(":", -1); // allow empty column specifier + if (values.length != 2 || !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"); @@ -280,7 +256,16 @@ { logger.debug("get_column_count"); String[] values = RowMutation.getColumnAndColumnFamily(columnParent); - ColumnFamily cfamily = readColumnFamily(new SliceReadCommand(tablename, key, columnParent, -1, Integer.MAX_VALUE)); + ColumnFamily cfamily; + + if (DatabaseDescriptor.isNameSortingEnabled(tablename, values[0])) + { + cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnParent + ":", true, Integer.MAX_VALUE)); + } + else + { + cfamily = readColumnFamily(new ColumnsSinceReadCommand(tablename, key, columnParent, Long.MIN_VALUE)); + } if (cfamily == null) { return 0; @@ -382,10 +367,15 @@ return thriftSuperColumns; } - public List<superColumn_t> get_slice_super(String tablename, String key, String columnFamily, int start, int count) throws InvalidRequestException + public List<superColumn_t> get_slice_super(String tablename, String key, String columnFamily, boolean isAscending, int count) throws InvalidRequestException { logger.debug("get_slice_super"); - ColumnFamily cfamily = readColumnFamily(new SliceReadCommand(tablename, key, columnFamily, start, count)); + 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"); + if (count <= 0) + throw new InvalidRequestException("get_slice_super requires positive count"); + ColumnFamily cfamily = readColumnFamily(new SliceFromReadCommand(tablename, key, columnFamily, isAscending, count)); if (cfamily == null) { return EMPTY_SUPERCOLUMNS; Modified: incubator/cassandra/trunk/test/system/test_server.py URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/test_server.py?rev=788516&r1=788515&r2=788516&view=diff ============================================================================== --- incubator/cassandra/trunk/test/system/test_server.py (original) +++ incubator/cassandra/trunk/test/system/test_server.py Thu Jun 25 21:26:33 2009 @@ -43,13 +43,13 @@ def _verify_batch(): _verify_simple() - L = client.get_slice('Table1', 'key1', 'Standard2', -1, -1) + L = client.get_slice('Table1', 'key1', 'Standard2:', True, 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', -1, -1) + L = client.get_slice('Table1', 'key1', 'Standard1:', True, 1000) assert L == _SIMPLE_COLUMNS, L def _insert_super(): @@ -80,7 +80,7 @@ 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', -1, -1) + slice = client.get_slice_super('Table1', 'key1', 'Super1:', True, 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', -1, -1) == [] + assert client.get_slice('Table1', 'key1', 'Standard2:', True, 1000) == [] def test_empty_slice_super(self): - assert client.get_slice('Table1', 'key1', 'Super1', -1, -1) == [] + assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 1000) == [] def test_missing_super(self): _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc1:c1')) @@ -158,24 +158,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', -1, -1) == \ + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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', -1, -1) == \ + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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: client.insert('Table1', 'key1', 'Standard1:c1', 'value1', 0, True) - assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == \ + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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', -1, -1) == \ + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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)] @@ -187,16 +187,16 @@ # Remove the key1:Standard1 cf: client.remove('Table1', 'key1', 'Standard1', 3, True) - assert client.get_slice('Table1', 'key1', 'Standard1', -1, -1) == [] + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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', -1, -1) == [] + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 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', -1, -1) == \ + assert client.get_slice('Table1', 'key1', 'Standard1:', True, 1000) == \ [column_t(columnName='c1', value='value1', timestamp=4)] @@ -207,7 +207,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', -1, -1) == \ + assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 1000) == \ [superColumn_t(name='sc1', columns=[column_t(columnName='c4', value='value4', timestamp=0)]), superColumn_t(name='sc2', @@ -222,17 +222,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', -1, -1) == scs + assert client.get_slice_super('Table1', 'key1', 'Super1:', True, 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', -1, -1) + actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 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', -1, -1) + actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 1000) assert actual == \ [superColumn_t(name='sc1', columns=[column_t(columnName='c4', value='value4', timestamp=0)]), @@ -248,23 +248,23 @@ # Make sure remove clears out what it's supposed to, and _only_ that: client.remove('Table1', 'key1', 'Super1:sc2', 5, True) _expect_missing(lambda: client.get_column('Table1', 'key1', 'Super1:sc2:c5')) - actual = client.get_slice('Table1', 'key1', 'Super1:sc2', -1, -1) + actual = client.get_columns_since('Table1', 'key1', 'Super1:sc2', -1) 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', -1, -1) + actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 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', -1, -1) + actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 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', -1, -1) + actual = client.get_slice_super('Table1', 'key1', 'Super1:', True, 1000) assert actual == \ [superColumn_t(name='sc1', columns=[column_t(columnName='c4', value='value4', timestamp=0)]), 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=788516&r1=788515&r2=788516&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 Thu Jun 25 21:26:33 2009 @@ -57,7 +57,7 @@ rm2 = serializeAndDeserializeReadMessage(rm); assert rm2.toString().equals(rm.toString()); - rm = new SliceReadCommand("Table1", "row1", "foo", 1, 2); + rm = new SliceFromReadCommand("Table1", "row1", "foo", true, 2); rm2 = serializeAndDeserializeReadMessage(rm); assert rm2.toString().equals(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=788516&r1=788515&r2=788516&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 Thu Jun 25 21:26:33 2009 @@ -80,92 +80,7 @@ }; reTest(setup, table.getColumnFamilyStore("Standard1"), verify); } - - @Test - public void testGetRowOffsetCount() throws Throwable - { - final Table table = Table.open("Table1"); - - Runner setup = new Runner() - { - public void run() throws Exception - { - RowMutation rm = makeSimpleRowMutation(); //inserts col1, col2, col3 - rm.apply(); - } - }; - Runner verify = new Runner() - { - public void run() throws Exception - { - Row result = table.getRow(TEST_KEY, "Standard1", 0, 2); - ColumnFamily cres = result.getColumnFamily("Standard1"); - assertNotNull(cres); - assertEquals(cres.getColumnCount(), 2); - // should have col1 and col2 - assertNotNull(cres.getColumn("col1")); - assertNotNull(cres.getColumn("col2")); - - result = table.getRow(TEST_KEY, "Standard1", 1, 2); - cres = result.getColumnFamily("Standard1"); - assertNotNull(cres); - assertEquals(2, cres.getColumnCount()); - // offset is 1, so we should have col2 and col3 - assertNotNull(cres.getColumn("col2")); - assertNotNull(cres.getColumn("col3")); - } - }; - reTest(setup, table.getColumnFamilyStore("Standard1"), verify); - } - - @Test - public void testGetRowOffsetCountFromDisk() throws Throwable - { - Table table = Table.open("Table1"); - - RowMutation rm = new RowMutation("Table1",KEY2); - ColumnFamily cf = ColumnFamily.create("Table1", "Standard1"); - // First write 5, 6 - cf.addColumn(new Column("col5", "val5".getBytes(), 1L)); - cf.addColumn(new Column("col6", "val6".getBytes(), 1L)); - rm.add(cf); - - rm.apply(); - Row result = table.getRow(KEY2, "Standard1:col5"); - ColumnFamily cres = result.getColumnFamily("Standard1"); - assertNotNull(cres.getColumn("col5")); - - table.getColumnFamilyStore("Standard1").forceBlockingFlush(); - // Flushed memtable to disk, we're now inserting into a new memtable - - rm = new RowMutation("Table1", KEY2); - cf = ColumnFamily.create("Table1", "Standard1"); - // now write 7, 8, 4 into new memtable - cf.addColumn(new Column("col7", "val7".getBytes(), 1L)); - cf.addColumn(new Column("col8", "val8".getBytes(), 1L)); - cf.addColumn(new Column("col4", "val4".getBytes(), 1L)); - rm.add(cf); - rm.apply(); - - // Check col5 is still around - result = table.getRow(KEY2, "Standard1:col5"); - cres = result.getColumnFamily("Standard1"); - assertNotNull(cres.getColumn("col5")); - - // Read back 3 cols from start -- should see 4,5,6 NOT 4,7,8 - result = table.getRow(KEY2, "Standard1", 0, 3); - cres = result.getColumnFamily("Standard1"); - assertNotNull(cres); - assertEquals(cres.getColumnCount(), 3); - - // Should have col4, col5, and col6 - assertNotNull(cres.getColumn("col4")); - assertNotNull(cres.getColumn("col5")); - assertNotNull(cres.getColumn("col6")); - // not 8 - assertNull(cres.getColumn("col8")); - } - + @Test public void testGetRowSliceByRange() throws Throwable { @@ -193,47 +108,6 @@ assertEquals(0, result.getColumnFamily("Standard1").getColumnCount()); } - - @Test - public void testGetRowSuperColumnOffsetCount() throws Throwable - { - Table table = Table.open("Table1"); - - RowMutation rm = new RowMutation("Table1",TEST_KEY); - ColumnFamily cf = ColumnFamily.create("Table1", "Super1"); - SuperColumn sc1 = new SuperColumn("sc1"); - sc1.addColumn(new Column("col1","val1".getBytes(), 1L)); - sc1.addColumn(new Column("col2","val2".getBytes(), 1L)); - SuperColumn sc2 = new SuperColumn("sc2"); - sc2.addColumn(new Column("col3","val3".getBytes(), 1L)); - sc2.addColumn(new Column("col4","val4".getBytes(), 1L)); - cf.addColumn(sc1); - cf.addColumn(sc2); - rm.add(cf); - - rm.apply(); - //Slicing top level columns of a supercolumn - Row result = table.getRow(TEST_KEY, "Super1", 0, 2); - ColumnFamily cres = result.getColumnFamily("Super1"); - assertNotNull(cres); - assertEquals(cres.getAllColumns().size(), 2); //2 supercolumns - assertEquals(cres.getColumnCount(),2+4); //2 cols, 2 subcols each - // should have col1 ... col4 - assertNotNull(cres.getColumn("sc1").getSubColumn("col1")); - assertNotNull(cres.getColumn("sc1").getSubColumn("col2")); - assertNotNull(cres.getColumn("sc2").getSubColumn("col3")); - assertNotNull(cres.getColumn("sc2").getSubColumn("col4")); - - - result = table.getRow(TEST_KEY, "Super1:sc1", 1, 2); //get at most 2, but only 1 column will be available - cres = result.getColumnFamily("Super1"); - assertNotNull(cres); - assertEquals(cres.getAllColumns().size(), 1); //only 1 top level column. sc1 has only 2 subcolumns - assertEquals(cres.getColumnCount(), 2); //getObjectCount: 1 for the column, and 1 for subcolumn - - assertNotNull(cres.getColumn("sc1").getSubColumn("col2")); - assertNull(cres.getColumn("sc2")); - } private RowMutation makeSimpleRowMutation() { @@ -272,12 +146,12 @@ ColumnFamily cf; // key before the rows that exists - result = table.getSliceFrom("a", "Standard2", true, 0); + result = table.getSliceFrom("a", "Standard2:", true, 0); cf = result.getColumnFamily("Standard2"); assertColumns(cf); // key after the rows that exist - result = table.getSliceFrom("z", "Standard2", true, 0); + result = table.getSliceFrom("z", "Standard2:", true, 0); cf = result.getColumnFamily("Standard2"); assertColumns(cf); } @@ -441,6 +315,43 @@ assertEquals(new String(cfres.getColumn("col1992").value()), "vvvvvvvvvvvvvvvv1992"); } + @Test + public void testGetSliceFromSuperBasic() throws Throwable + { + // tests slicing against data from one row spread across two sstables + final Table table = Table.open("Table1"); + final String ROW = "row2"; + Runner setup = new Runner() + { + public void run() throws Exception + { + RowMutation rm = new RowMutation("Table1", ROW); + ColumnFamily cf = ColumnFamily.create("Table1", "Super1"); + SuperColumn sc = new SuperColumn("sc1"); + sc.addColumn(new Column("col1", "val1".getBytes(), 1L)); + cf.addColumn(sc); + rm.add(cf); + rm.apply(); + } + }; + + Runner verify = new Runner() + { + public void run() throws Exception + { + Row result; + ColumnFamily cfres; + + result = table.getSliceFrom(ROW, "Super1:", true, 10); + cfres = result.getColumnFamily("Super1"); + assertColumns(cfres, "sc1"); + assertEquals(new String(cfres.getColumn("sc1").getSubColumn("col1").value()), "val1"); + } + }; + + reTest(setup, table.getColumnFamilyStore("Standard1"), verify); + } + public static void assertColumns(ColumnFamily columnFamily, String... columnNames) { assertNotNull(columnFamily); @@ -453,5 +364,4 @@ assert Arrays.equals(L.toArray(new String[columns.size()]), columnNames) : "Columns [" + StringUtils.join(columns, ", ") + "] is not expected [" + StringUtils.join(columnNames, ", ") + "]"; } - }
