@Test public void testSuperColumnTombstones() throws IOException, ExecutionException, InterruptedException { Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Super1"); cfs.disableAutoCompaction(); DecoratedKey key = Util.dk("tskey"); ByteBuffer scName = ByteBufferUtil.bytes("TestSuperColumn"); // a subcolumn RowMutation rm = new RowMutation(KEYSPACE1, key.key); rm.add( "Super1", CompositeType.build(scName, ByteBufferUtil.bytes(0)), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()); rm.apply(); cfs.forceBlockingFlush(); // shadow the subcolumn with a supercolumn tombstone rm = new RowMutation(KEYSPACE1, key.key); rm.deleteRange( "Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros()); rm.apply(); cfs.forceBlockingFlush(); CompactionManager.instance.performMaximal(cfs); assertEquals(1, cfs.getSSTables().size()); // check that the shadowed column is gone SSTableReader sstable = cfs.getSSTables().iterator().next(); Range keyRange = new Range<RowPosition>(key, sstable.partitioner.getMinimumToken().maxKeyBound()); SSTableScanner scanner = sstable.getScanner(DataRange.forKeyRange(keyRange)); OnDiskAtomIterator iter = scanner.next(); assertEquals(key, iter.getKey()); assert iter.next() instanceof RangeTombstone; assert !iter.hasNext(); }
@Test public void testNoExpire() throws ExecutionException, InterruptedException { ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1"); cfs.disableAutoCompaction(); cfs.metadata.gcGraceSeconds(0); long timestamp = System.currentTimeMillis(); RowMutation rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add( "Standard1", ByteBufferUtil.bytes("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.add( "Standard1", ByteBufferUtil.bytes("col7"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add( "Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); rm = new RowMutation("Keyspace1", Util.dk("ttl").key); rm.add( "Standard1", ByteBufferUtil.bytes("col3"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1); rm.apply(); cfs.forceBlockingFlush(); DecoratedKey noTTLKey = Util.dk("nottl"); rm = new RowMutation("Keyspace1", noTTLKey.key); rm.add( "Standard1", ByteBufferUtil.bytes("col311"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp); rm.apply(); cfs.forceBlockingFlush(); Thread.sleep(2000); // wait for ttl to expire assertEquals(4, cfs.getSSTables().size()); cfs.enableAutoCompaction(true); assertEquals(1, cfs.getSSTables().size()); SSTableReader sstable = cfs.getSSTables().iterator().next(); SSTableScanner scanner = sstable.getScanner(DataRange.allData(sstable.partitioner)); assertTrue(scanner.hasNext()); while (scanner.hasNext()) { OnDiskAtomIterator iter = scanner.next(); assertEquals(noTTLKey, iter.getKey()); } }