public ColumnFamilyStore testSingleSSTableCompaction(String strategyClassName) throws Exception {
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
    store.clearUnsafe();
    store.metadata.gcGraceSeconds(1);
    store.setCompactionStrategyClass(strategyClassName);

    // disable compaction while flushing
    store.disableAutoCompaction();

    long timestamp = populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); // ttl=3s

    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;
  }
  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 testAggressiveFullyExpired() {
    String KEYSPACE1 = "Keyspace1";
    ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
    cfs.disableAutoCompaction();
    cfs.metadata.gcGraceSeconds(0);

    DecoratedKey ttlKey = Util.dk("ttl");
    RowMutation rm = new RowMutation("Keyspace1", ttlKey.key);
    rm.add("Standard1", ByteBufferUtil.bytes("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 1, 1);
    rm.add("Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 3, 1);
    rm.applyUnsafe();
    cfs.forceBlockingFlush();

    rm = new RowMutation(KEYSPACE1, ttlKey.key);
    rm.add("Standard1", ByteBufferUtil.bytes("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2, 1);
    rm.add("Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 5, 1);
    rm.applyUnsafe();
    cfs.forceBlockingFlush();

    rm = new RowMutation(KEYSPACE1, ttlKey.key);
    rm.add("Standard1", ByteBufferUtil.bytes("col1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 4, 1);
    rm.add("Standard1", ByteBufferUtil.bytes("shadow"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 7, 1);
    rm.applyUnsafe();
    cfs.forceBlockingFlush();

    rm = new RowMutation(KEYSPACE1, ttlKey.key);
    rm.add("Standard1", ByteBufferUtil.bytes("shadow"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 6, 3);
    rm.add("Standard1", ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8, 1);
    rm.applyUnsafe();
    cfs.forceBlockingFlush();

    Set<SSTableReader> sstables = Sets.newHashSet(cfs.getSSTables());
    int now = (int) (System.currentTimeMillis() / 1000);
    int gcBefore = now + 2;
    Set<SSTableReader> expired =
        CompactionController.getFullyExpiredSSTables(
            cfs, sstables, Collections.EMPTY_SET, gcBefore);
    assertEquals(2, expired.size());

    cfs.clearUnsafe();
  }
  @Test
  @Ignore("making ranges based on the keys, not on the tokens")
  public void testNeedsCleanup() {
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore store = keyspace.getColumnFamilyStore("CF_STANDARD1");
    store.clearUnsafe();

    // disable compaction while flushing
    store.disableAutoCompaction();

    // write three groups of 9 keys: 001, 002, ... 008, 009
    //                               101, 102, ... 108, 109
    //                               201, 202, ... 208, 209
    for (int i = 1; i < 10; i++) {
      insertRowWithKey(i);
      insertRowWithKey(i + 100);
      insertRowWithKey(i + 200);
    }
    store.forceBlockingFlush();

    assertEquals(1, store.getSSTables().size());
    SSTableReader sstable = store.getSSTables().iterator().next();

    // contiguous range spans all data
    assertFalse(CompactionManager.needsCleanup(sstable, makeRanges(0, 209)));
    assertFalse(CompactionManager.needsCleanup(sstable, makeRanges(0, 210)));

    // separate ranges span all data
    assertFalse(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 109,
                200, 209)));
    assertFalse(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 109,
                200, 210)));
    assertFalse(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 210)));

    // one range is missing completely
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                100, 109,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 109)));

    // the beginning of one range is missing
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                1, 9,
                100, 109,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                101, 109,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 109,
                201, 209)));

    // the end of one range is missing
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 8,
                100, 109,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 108,
                200, 209)));
    assertTrue(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 9,
                100, 109,
                200, 208)));

    // some ranges don't contain any data
    assertFalse(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 0,
                0, 9,
                50, 51,
                100, 109,
                150, 199,
                200, 209,
                300, 301)));
    // same case, but with a middle range not covering some of the existing data
    assertFalse(
        CompactionManager.needsCleanup(
            sstable,
            makeRanges(
                0, 0,
                0, 9,
                50, 51,
                100, 103,
                150, 199,
                200, 209,
                300, 301)));
  }
  @Test
  public void testRangeTombstones() throws IOException, ExecutionException, InterruptedException {
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard2");
    cfs.clearUnsafe();

    // disable compaction while flushing
    cfs.disableAutoCompaction();

    final CFMetaData cfmeta = cfs.metadata;
    Directories dir = cfs.directories;

    ArrayList<DecoratedKey> keys = new ArrayList<DecoratedKey>();

    for (int i = 0; i < 4; i++) {
      keys.add(Util.dk("" + i));
    }

    ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfmeta);
    cf.addColumn(Util.column("01", "a", 1)); // this must not resurrect
    cf.addColumn(Util.column("a", "a", 3));
    cf.deletionInfo()
        .add(
            new RangeTombstone(
                Util.cellname("0"),
                Util.cellname("b"),
                2,
                (int) (System.currentTimeMillis() / 1000)),
            cfmeta.comparator);

    SSTableWriter writer =
        new SSTableWriter(
            cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables()),
            0,
            0,
            cfs.metadata,
            StorageService.getPartitioner(),
            new MetadataCollector(cfs.metadata.comparator));

    writer.append(Util.dk("0"), cf);
    writer.append(Util.dk("1"), cf);
    writer.append(Util.dk("3"), cf);

    cfs.addSSTable(writer.closeAndOpenReader());
    writer =
        new SSTableWriter(
            cfs.getTempSSTablePath(dir.getDirectoryForNewSSTables()),
            0,
            0,
            cfs.metadata,
            StorageService.getPartitioner(),
            new MetadataCollector(cfs.metadata.comparator));

    writer.append(Util.dk("0"), cf);
    writer.append(Util.dk("1"), cf);
    writer.append(Util.dk("2"), cf);
    writer.append(Util.dk("3"), cf);
    cfs.addSSTable(writer.closeAndOpenReader());

    Collection<SSTableReader> toCompact = cfs.getSSTables();
    assert toCompact.size() == 2;

    // Force compaction on first sstables. Since each row is in only one sstable, we will be using
    // EchoedRow.
    Util.compact(cfs, toCompact);
    assertEquals(1, cfs.getSSTables().size());

    // Now assert we do have the 4 keys
    assertEquals(4, Util.getRangeSlice(cfs).size());

    ArrayList<DecoratedKey> k = new ArrayList<DecoratedKey>();
    for (Row r : Util.getRangeSlice(cfs)) {
      k.add(r.key);
      assertEquals(ByteBufferUtil.bytes("a"), r.cf.getColumn(Util.cellname("a")).value());
      assertNull(r.cf.getColumn(Util.cellname("01")));
      assertEquals(3, r.cf.getColumn(Util.cellname("a")).timestamp());
    }

    for (SSTableReader sstable : cfs.getSSTables()) {
      StatsMetadata stats = sstable.getSSTableMetadata();
      assertEquals(ByteBufferUtil.bytes("0"), stats.minColumnNames.get(0));
      assertEquals(ByteBufferUtil.bytes("b"), stats.maxColumnNames.get(0));
    }

    assertEquals(keys, k);
  }
  @Test
  public void testUncheckedTombstoneSizeTieredCompaction() throws Exception {
    Keyspace keyspace = Keyspace.open(KEYSPACE1);
    ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
    store.clearUnsafe();
    store.metadata.gcGraceSeconds(1);
    store.metadata.compactionStrategyOptions.put("tombstone_compaction_interval", "1");
    store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "false");
    store.reload();
    store.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getName());

    // disable compaction while flushing
    store.disableAutoCompaction();

    // Populate sstable1 with with keys [0..9]
    populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); // ttl=3s
    store.forceBlockingFlush();

    // Populate sstable2 with with keys [10..19] (keys do not overlap with SSTable1)
    long timestamp2 = populate(KEYSPACE1, CF_STANDARD1, 10, 19, 3); // ttl=3s
    store.forceBlockingFlush();

    assertEquals(2, store.getSSTables().size());

    Iterator<SSTableReader> it = store.getSSTables().iterator();
    long originalSize1 = it.next().uncompressedLength();
    long originalSize2 = it.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);

    // even though both sstables were candidate for tombstone compaction
    // it was not executed because they have an overlapping token range
    assertEquals(2, store.getSSTables().size());
    it = store.getSSTables().iterator();
    long newSize1 = it.next().uncompressedLength();
    long newSize2 = it.next().uncompressedLength();
    assertEquals(
        "candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
        originalSize1,
        newSize1);
    assertEquals(
        "candidate sstable should not be tombstone-compacted because its key range overlap with other sstable",
        originalSize2,
        newSize2);

    // now let's enable the magic property
    store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "true");
    store.reload();

    // submit background task again and wait for it to complete
    FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(store));
    while (CompactionManager.instance.getPendingTasks() > 0
        || CompactionManager.instance.getActiveCompactions() > 0) TimeUnit.SECONDS.sleep(1);

    // we still have 2 sstables, since they were not compacted against each other
    assertEquals(2, store.getSSTables().size());
    it = store.getSSTables().iterator();
    newSize1 = it.next().uncompressedLength();
    newSize2 = it.next().uncompressedLength();
    assertTrue(
        "should be less than " + originalSize1 + ", but was " + newSize1, newSize1 < originalSize1);
    assertTrue(
        "should be less than " + originalSize2 + ", but was " + newSize2, newSize2 < originalSize2);

    // make sure max timestamp of compacted sstables is recorded properly after compaction.
    assertMaxTimestamp(store, timestamp2);
  }