private static void insertRowWithKey(int key) { long timestamp = System.currentTimeMillis(); DecoratedKey decoratedKey = Util.dk(String.format("%03d", key)); Mutation rm = new Mutation(KEYSPACE1, decoratedKey.getKey()); rm.add("CF_STANDARD1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000); rm.applyUnsafe(); }
/** * Writes out a bunch of mutations for a single column family. * * @param mutations A group of Mutations for the same keyspace and column family. * @return The ColumnFamilyStore that was used. */ public static ColumnFamilyStore writeColumnFamily(List<Mutation> mutations) { IMutation first = mutations.get(0); String keyspaceName = first.getKeyspaceName(); UUID cfid = first.getColumnFamilyIds().iterator().next(); for (Mutation rm : mutations) rm.applyUnsafe(); ColumnFamilyStore store = Keyspace.open(keyspaceName).getColumnFamilyStore(cfid); store.forceBlockingFlush(); return store; }
private void testDontPurgeAccidentaly(String k, String cfname) throws 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); Mutation rm = new Mutation(KEYSPACE1, key.getKey()); rm.add( cfname, Util.cellname(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0); rm.applyUnsafe(); cfs.forceBlockingFlush(); Collection<SSTableReader> sstablesBefore = cfs.getSSTables(); QueryFilter filter = QueryFilter.getIdentityFilter(key, cfname, System.currentTimeMillis()); assertTrue(cfs.getColumnFamily(filter).hasColumns()); // Remove key rm = new Mutation(KEYSPACE1, key.getKey()); rm.delete(cfname, 2); rm.applyUnsafe(); ColumnFamily cf = cfs.getColumnFamily(filter); assertTrue("should be empty: " + cf, cf == null || !cf.hasColumns()); // 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); assertTrue("should be empty: " + cf, cf == null || !cf.hasColumns()); }
private long populate(String ks, String cf, int startRowKey, int endRowKey, int ttl) { long timestamp = System.currentTimeMillis(); for (int i = startRowKey; i <= endRowKey; i++) { DecoratedKey key = Util.dk(Integer.toString(i)); Mutation rm = new Mutation(ks, key.getKey()); for (int j = 0; j < 10; j++) rm.add( cf, Util.cellname(Integer.toString(j)), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, j > 0 ? ttl : 0); // let first column never expire, since deleting all columns does not produce // sstable rm.applyUnsafe(); } return timestamp; }
@Test public void testSuperColumnTombstones() throws 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 Mutation rm = new Mutation(KEYSPACE1, key.getKey()); rm.add( "Super1", Util.cellname(scName, ByteBufferUtil.bytes(0)), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()); rm.applyUnsafe(); cfs.forceBlockingFlush(); // shadow the subcolumn with a supercolumn tombstone rm = new Mutation(KEYSPACE1, key.getKey()); rm.deleteRange( "Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros()); rm.applyUnsafe(); 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()); assertTrue(iter.next() instanceof RangeTombstone); assertFalse(iter.hasNext()); }
@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)); Mutation rm = new Mutation(KEYSPACE1, key.getKey()); rm.add( cfname, Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis()); rm.applyUnsafe(); } cfs.forceBlockingFlush(); Collection<SSTableReader> sstables = cfs.getSSTables(); assertEquals(1, sstables.size()); SSTableReader sstable = sstables.iterator().next(); int prevGeneration = sstable.descriptor.generation; String file = new File(sstable.descriptor.filenameFor(Component.DATA)).getAbsolutePath(); // 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(); assertEquals(1, sstables.size()); assertEquals(prevGeneration + 1, sstables.iterator().next().descriptor.generation); }
public static void addMutation( Mutation rm, String columnFamilyName, String superColumnName, long columnName, String value, long timestamp) { CellName cname = superColumnName == null ? CellNames.simpleDense(getBytes(columnName)) : CellNames.compositeDense(ByteBufferUtil.bytes(superColumnName), getBytes(columnName)); rm.add(columnFamilyName, cname, ByteBufferUtil.bytes(value), timestamp); }
@Test public void testEchoedRow() { // 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)); Mutation rm = new Mutation(KEYSPACE1, key.getKey()); rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i); rm.applyUnsafe(); if (i % 2 == 0) cfs.forceBlockingFlush(); } Collection<SSTableReader> toCompact = cfs.getSSTables(); assertEquals(2, toCompact.size()); // 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)); Mutation rm = new Mutation(KEYSPACE1, key.getKey()); rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i); rm.applyUnsafe(); } cfs.forceBlockingFlush(); SSTableReader tmpSSTable = null; for (SSTableReader sstable : cfs.getSSTables()) if (!toCompact.contains(sstable)) tmpSSTable = sstable; assertNotNull(tmpSSTable); // 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()); }