private static void insertRowWithKey(int key) { long timestamp = System.currentTimeMillis(); DecoratedKey decoratedKey = Util.dk(String.format("%03d", key)); RowMutation rm = new RowMutation(KEYSPACE1, decoratedKey.key); rm.add( "Standard1", ByteBufferUtil.bytes("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000); rm.apply(); }
private void testDontPurgeAccidentaly(String k, String cfname) throws IOException, ExecutionException, InterruptedException { // This test catches the regression of CASSANDRA-2786 Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname); // disable compaction while flushing cfs.clearUnsafe(); cfs.disableAutoCompaction(); // Add test row DecoratedKey key = Util.dk(k); RowMutation rm = new RowMutation(KEYSPACE1, key.key); rm.add( cfname, CompositeType.build(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0); rm.apply(); cfs.forceBlockingFlush(); Collection<SSTableReader> sstablesBefore = cfs.getSSTables(); QueryFilter filter = QueryFilter.getIdentityFilter(key, cfname, System.currentTimeMillis()); assert !(cfs.getColumnFamily(filter).getColumnCount() == 0); // Remove key rm = new RowMutation(KEYSPACE1, key.key); rm.delete(cfname, 2); rm.apply(); ColumnFamily cf = cfs.getColumnFamily(filter); assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + cf; // Sleep one second so that the removal is indeed purgeable even with gcgrace == 0 Thread.sleep(1000); cfs.forceBlockingFlush(); Collection<SSTableReader> sstablesAfter = cfs.getSSTables(); Collection<SSTableReader> toCompact = new ArrayList<SSTableReader>(); for (SSTableReader sstable : sstablesAfter) if (!sstablesBefore.contains(sstable)) toCompact.add(sstable); Util.compact(cfs, toCompact); cf = cfs.getColumnFamily(filter); assert cf == null || cf.getColumnCount() == 0 : "should be empty: " + cf; }
public ColumnFamilyStore testSingleSSTableCompaction(String strategyClassName) throws Exception { Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1"); store.clearUnsafe(); store.metadata.gcGraceSeconds(1); store.setCompactionStrategyClass(strategyClassName); // disable compaction while flushing store.disableAutoCompaction(); long timestamp = System.currentTimeMillis(); for (int i = 0; i < 10; i++) { DecoratedKey key = Util.dk(Integer.toString(i)); RowMutation rm = new RowMutation(KEYSPACE1, key.key); for (int j = 0; j < 10; j++) rm.add( "Standard1", ByteBufferUtil.bytes(Integer.toString(j)), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, j > 0 ? 3 : 0); // let first column never expire, since deleting all columns does not produce // sstable rm.apply(); } store.forceBlockingFlush(); assertEquals(1, store.getSSTables().size()); long originalSize = store.getSSTables().iterator().next().uncompressedLength(); // wait enough to force single compaction TimeUnit.SECONDS.sleep(5); // enable compaction, submit background and wait for it to complete store.enableAutoCompaction(); FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(store)); while (CompactionManager.instance.getPendingTasks() > 0 || CompactionManager.instance.getActiveCompactions() > 0) TimeUnit.SECONDS.sleep(1); // and sstable with ttl should be compacted assertEquals(1, store.getSSTables().size()); long size = store.getSSTables().iterator().next().uncompressedLength(); assertTrue("should be less than " + originalSize + ", but was " + size, size < originalSize); // make sure max timestamp of compacted sstables is recorded properly after compaction. assertMaxTimestamp(store, timestamp); return store; }
@Test public void testGetColumn() throws IOException, ColumnFamilyNotDefinedException { Table table = Table.open("Keyspace1"); RowMutation rm; // add data rm = new RowMutation("Keyspace1", "key1"); rm.add(new QueryPath("Standard1", null, "Column1".getBytes()), "abcd".getBytes(), 0); rm.apply(); ReadCommand command = new SliceByNamesReadCommand( "Keyspace1", "key1", new QueryPath("Standard1"), Arrays.asList("Column1".getBytes())); Row row = command.getRow(table); IColumn col = row.cf.getColumn("Column1".getBytes()); assert Arrays.equals(col.value(), "abcd".getBytes()); }
@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(); }
/* * This excercise in particular the code of #4142 */ @Test public void testValidationMultipleSSTablePerLevel() throws Exception { String ksname = "Keyspace1"; String cfname = "StandardLeveled"; Table table = Table.open(ksname); ColumnFamilyStore store = table.getColumnFamilyStore(cfname); ByteBuffer value = ByteBuffer.wrap(new byte[100 * 1024]); // 100 KB value, make it easy to have multiple files // Enough data to have a level 1 and 2 int rows = 20; int columns = 10; // Adds enough data to trigger multiple sstable per level for (int r = 0; r < rows; r++) { DecoratedKey key = Util.dk(String.valueOf(r)); RowMutation rm = new RowMutation(ksname, key.key); for (int c = 0; c < columns; c++) { rm.add(new QueryPath(cfname, null, ByteBufferUtil.bytes("column" + c)), value, 0); } rm.apply(); store.forceFlush(); } LeveledCompactionStrategy strat = (LeveledCompactionStrategy) store.getCompactionStrategy(); while (strat.getLevelSize(0) > 0) { store.forceMajorCompaction(); Thread.sleep(200); } // Checking we're not completely bad at math assert strat.getLevelSize(1) > 0; assert strat.getLevelSize(2) > 0; AntiEntropyService.CFPair p = new AntiEntropyService.CFPair(ksname, cfname); Range<Token> range = new Range<Token>(Util.token(""), Util.token("")); AntiEntropyService.TreeRequest req = new AntiEntropyService.TreeRequest("1", FBUtilities.getLocalAddress(), range, p); AntiEntropyService.Validator validator = new AntiEntropyService.Validator(req); CompactionManager.instance.submitValidation(store, validator).get(); }
@Test public void testUserDefinedCompaction() throws Exception { Keyspace keyspace = Keyspace.open(KEYSPACE1); final String cfname = "Standard3"; // use clean(no sstable) CF ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname); // disable compaction while flushing cfs.disableAutoCompaction(); final int ROWS_PER_SSTABLE = 10; for (int i = 0; i < ROWS_PER_SSTABLE; i++) { DecoratedKey key = Util.dk(String.valueOf(i)); RowMutation rm = new RowMutation(KEYSPACE1, key.key); rm.add( cfname, ByteBufferUtil.bytes("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis()); rm.apply(); } cfs.forceBlockingFlush(); Collection<SSTableReader> sstables = cfs.getSSTables(); assert sstables.size() == 1; SSTableReader sstable = sstables.iterator().next(); int prevGeneration = sstable.descriptor.generation; String file = new File(sstable.descriptor.filenameFor(Component.DATA)).getName(); // submit user defined compaction on flushed sstable CompactionManager.instance.forceUserDefinedCompaction(file); // wait until user defined compaction finishes do { Thread.sleep(100); } while (CompactionManager.instance.getPendingTasks() > 0 || CompactionManager.instance.getActiveCompactions() > 0); // CF should have only one sstable with generation number advanced sstables = cfs.getSSTables(); assert sstables.size() == 1; assert sstables.iterator().next().descriptor.generation == prevGeneration + 1; }
@Test public void testEchoedRow() throws IOException, ExecutionException, InterruptedException { // This test check that EchoedRow doesn't skipp rows: see CASSANDRA-2653 Keyspace keyspace = Keyspace.open(KEYSPACE1); ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard2"); // disable compaction while flushing cfs.disableAutoCompaction(); // Insert 4 keys in two sstables. We need the sstables to have 2 rows // at least to trigger what was causing CASSANDRA-2653 for (int i = 1; i < 5; i++) { DecoratedKey key = Util.dk(String.valueOf(i)); RowMutation rm = new RowMutation(KEYSPACE1, key.key); rm.add( "Standard2", ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i); rm.apply(); if (i % 2 == 0) cfs.forceBlockingFlush(); } Collection<SSTableReader> toCompact = cfs.getSSTables(); assert toCompact.size() == 2; // Reinserting the same keys. We will compact only the previous sstable, but we need those new // ones // to make sure we use EchoedRow, otherwise it won't be used because purge can be done. for (int i = 1; i < 5; i++) { DecoratedKey key = Util.dk(String.valueOf(i)); RowMutation rm = new RowMutation(KEYSPACE1, key.key); rm.add( "Standard2", ByteBufferUtil.bytes(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i); rm.apply(); } cfs.forceBlockingFlush(); SSTableReader tmpSSTable = null; for (SSTableReader sstable : cfs.getSSTables()) if (!toCompact.contains(sstable)) tmpSSTable = sstable; assert tmpSSTable != null; // Force compaction on first sstables. Since each row is in only one sstable, we will be using // EchoedRow. Util.compact(cfs, toCompact); assertEquals(2, cfs.getSSTables().size()); // Now, we remove the sstable that was just created to force the use of EchoedRow (so that it // doesn't hide the problem) cfs.markObsolete(Collections.singleton(tmpSSTable), OperationType.UNKNOWN); assertEquals(1, cfs.getSSTables().size()); // Now assert we do have the 4 keys assertEquals(4, Util.getRangeSlice(cfs).size()); }