You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/05/22 10:46:32 UTC

[1/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Repository: cassandra
Updated Branches:
  refs/heads/trunk 15d424e86 -> d96a02a12


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 5dca589..fa91d00 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -18,14 +18,16 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.File;
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.junit.After;
 import org.junit.BeforeClass;
+import com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -43,6 +45,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.metrics.StorageMetrics;
@@ -52,7 +55,6 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 import static org.junit.Assert.*;
-import static org.apache.cassandra.utils.Throwables.maybeFail;
 
 public class SSTableRewriterTest extends SchemaLoader
 {
@@ -83,7 +85,9 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+
         for (int j = 0; j < 100; j ++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
@@ -94,8 +98,10 @@ public class SSTableRewriterTest extends SchemaLoader
         cfs.forceBlockingFlush();
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         assertEquals(1, sstables.size());
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
-             AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+        assertEquals(sstables.iterator().next().bytesOnDisk(), cfs.metric.liveDiskSpaceUsed.getCount());
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -105,30 +111,29 @@ public class SSTableRewriterTest extends SchemaLoader
                 AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
                 writer.append(row);
             }
-            Collection<SSTableReader> newsstables = writer.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables , OperationType.COMPACTION);
+            writer.finish();
         }
         SSTableDeletingTask.waitForDeletions();
-
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
-
+        truncate(cfs);
     }
     @Test
     public void basicTest2() throws InterruptedException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         assertEquals(1, sstables.size());
         SSTableRewriter.overrideOpenInterval(10000000);
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
-             AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -138,11 +143,9 @@ public class SSTableRewriterTest extends SchemaLoader
                 AbstractCompactedRow row = new LazilyCompactedRow(controller, Arrays.asList(scanner.next()));
                 writer.append(row);
             }
-            Collection<SSTableReader> newsstables = writer.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
+            writer.finish();
         }
         SSTableDeletingTask.waitForDeletions();
-
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
@@ -153,7 +156,9 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
@@ -161,8 +166,9 @@ public class SSTableRewriterTest extends SchemaLoader
         assertEquals(1, sstables.size());
         SSTableRewriter.overrideOpenInterval(10000000);
         boolean checked = false;
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
-             AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);)
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             ISSTableScanner scanner = scanners.scanners.get(0);
             CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
@@ -178,7 +184,7 @@ public class SSTableRewriterTest extends SchemaLoader
                     {
                         if (sstable.openReason == SSTableReader.OpenReason.EARLY)
                         {
-                            SSTableReader c = sstables.iterator().next();
+                            SSTableReader c = txn.current(sstables.iterator().next());
                             Collection<Range<Token>> r = Arrays.asList(new Range<>(cfs.partitioner.getMinimumToken(), cfs.partitioner.getMinimumToken()));
                             List<Pair<Long, Long>> tmplinkPositions = sstable.getPositionsForRanges(r);
                             List<Pair<Long, Long>> compactingPositions = c.getPositionsForRanges(r);
@@ -193,17 +199,14 @@ public class SSTableRewriterTest extends SchemaLoader
                 }
             }
             assertTrue(checked);
-            Collection<SSTableReader> newsstables = writer.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newsstables, OperationType.COMPACTION);
+            writer.finish();
         }
         SSTableDeletingTask.waitForDeletions();
-
         validateCFS(cfs);
         int filecounts = assertFileCounts(sstables.iterator().next().descriptor.directory.list(), 0, 0);
         assertEquals(1, filecounts);
-        cfs.truncateBlocking();
+        truncate(cfs);
         SSTableDeletingTask.waitForDeletions();
-
         validateCFS(cfs);
     }
 
@@ -212,7 +215,8 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
         ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         for (int i = 0; i < 100; i++)
             cf.addColumn(Util.cellname(i), ByteBuffer.allocate(1000), 1);
@@ -253,7 +257,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
@@ -263,9 +267,10 @@ public class SSTableRewriterTest extends SchemaLoader
 
         List<SSTableReader> sstables;
         int files = 1;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
@@ -282,8 +287,6 @@ public class SSTableRewriterTest extends SchemaLoader
                 }
             }
             sstables = rewriter.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
-            assertEquals(files, sstables.size());
         }
         long sum = 0;
         for (SSTableReader x : cfs.getSSTables())
@@ -305,7 +308,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
@@ -315,9 +318,10 @@ public class SSTableRewriterTest extends SchemaLoader
 
         List<SSTableReader> sstables;
         int files = 1;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
@@ -335,10 +339,6 @@ public class SSTableRewriterTest extends SchemaLoader
 
         assertEquals(files, sstables.size());
         assertEquals(files, cfs.getSSTables().size());
-        assertEquals(1, cfs.getDataTracker().getView().shadowed.size());
-        cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
-        assertEquals(files, cfs.getSSTables().size());
-        assertEquals(0, cfs.getDataTracker().getView().shadowed.size());
         SSTableDeletingTask.waitForDeletions();
 
         assertFileCounts(s.descriptor.directory.list(), 0, 0);
@@ -429,7 +429,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
@@ -440,9 +440,10 @@ public class SSTableRewriterTest extends SchemaLoader
         Set<SSTableReader> compacting = Sets.newHashSet(s);
         SSTableRewriter.overrideOpenInterval(10000000);
 
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             test.run(scanner, controller, s, cfs, rewriter);
@@ -463,7 +464,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
@@ -472,9 +473,10 @@ public class SSTableRewriterTest extends SchemaLoader
         SSTableRewriter.overrideOpenInterval(10000000);
 
         int files = 1;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
@@ -489,8 +491,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 if (files == 3)
                 {
                     //testing to finish when we have nothing written in the new file
-                    List<SSTableReader> sstables = rewriter.finish();
-                    cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
+                    rewriter.finish();
                     break;
                 }
             }
@@ -508,7 +509,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
         cfs.disableAutoCompaction();
 
         SSTableReader s = writeFile(cfs, 1000);
@@ -518,9 +519,10 @@ public class SSTableRewriterTest extends SchemaLoader
 
         List<SSTableReader> sstables;
         int files = 1;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
@@ -535,7 +537,6 @@ public class SSTableRewriterTest extends SchemaLoader
             }
 
             sstables = rewriter.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
         }
 
         SSTableDeletingTask.waitForDeletions();
@@ -550,7 +551,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
         cfs.disableAutoCompaction();
 
         SSTableReader s = writeFile(cfs, 400);
@@ -560,9 +561,10 @@ public class SSTableRewriterTest extends SchemaLoader
 
         List<SSTableReader> sstables;
         int files = 1;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = s.getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = s.getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);)
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while(scanner.hasNext())
@@ -577,7 +579,6 @@ public class SSTableRewriterTest extends SchemaLoader
             }
 
             sstables = rewriter.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, sstables, OperationType.COMPACTION);
         }
 
         assertEquals(files, sstables.size());
@@ -593,22 +594,24 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
         cfs.disableAutoCompaction();
         SSTableReader s = writeFile(cfs, 1000);
-        cfs.getDataTracker().markCompacting(Arrays.asList(s), true, false);
-        SSTableSplitter splitter = new SSTableSplitter(cfs, s, 10);
-        splitter.split();
+        try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UNKNOWN, s))
+        {
+            SSTableSplitter splitter = new SSTableSplitter(cfs, txn, 10);
+            splitter.split();
 
-        assertFileCounts(s.descriptor.directory.list(), 0, 0);
+            assertFileCounts(s.descriptor.directory.list(), 0, 0);
 
-        s.selfRef().release();
-        SSTableDeletingTask.waitForDeletions();
+            s.selfRef().release();
+            SSTableDeletingTask.waitForDeletions();
 
-        for (File f : s.descriptor.directory.listFiles())
-        {
-            // we need to clear out the data dir, otherwise tests running after this breaks
-            FileUtils.deleteRecursive(f);
+            for (File f : s.descriptor.directory.listFiles())
+            {
+                // we need to clear out the data dir, otherwise tests running after this breaks
+                FileUtils.deleteRecursive(f);
+            }
         }
     }
 
@@ -639,17 +642,18 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
         SSTableReader s = writeFile(cfs, 1000);
         if (!offline)
             cfs.addSSTable(s);
         Set<SSTableReader> compacting = Sets.newHashSet(s);
-        cfs.getDataTracker().markCompacting(compacting);
         SSTableRewriter.overrideOpenInterval(10000000);
-
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, offline);
-             ISSTableScanner scanner = compacting.iterator().next().getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = offline ? LifecycleTransaction.offline(OperationType.UNKNOWN, compacting)
+                                       : cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, offline);
+        )
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while (scanner.hasNext())
@@ -672,7 +676,6 @@ public class SSTableRewriterTest extends SchemaLoader
         }
         finally
         {
-            cfs.getDataTracker().unmarkCompacting(compacting);
             if (offline)
                 s.selfRef().release();
         }
@@ -686,9 +689,8 @@ public class SSTableRewriterTest extends SchemaLoader
             assertEquals(1, cfs.getSSTables().size());
             validateCFS(cfs);
         }
-        cfs.truncateBlocking();
+        truncate(cfs);
         SSTableDeletingTask.waitForDeletions();
-
         filecount = assertFileCounts(s.descriptor.directory.list(), 0, 0);
         if (offline)
         {
@@ -709,7 +711,7 @@ public class SSTableRewriterTest extends SchemaLoader
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        cfs.truncateBlocking();
+        truncate(cfs);
         for (int i = 0; i < 100; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
@@ -726,14 +728,14 @@ public class SSTableRewriterTest extends SchemaLoader
         SSTableReader s = cfs.getSSTables().iterator().next();
         Set<SSTableReader> compacting = new HashSet<>();
         compacting.add(s);
-        cfs.getDataTracker().markCompacting(compacting);
-
 
         SSTableRewriter.overrideOpenInterval(1);
         int keyCount = 0;
-        try (SSTableRewriter rewriter = new SSTableRewriter(cfs, compacting, 1000, false);
-             ISSTableScanner scanner = compacting.iterator().next().getScanner();
-             CompactionController controller = new CompactionController(cfs, compacting, 0))
+        try (ISSTableScanner scanner = compacting.iterator().next().getScanner();
+             CompactionController controller = new CompactionController(cfs, compacting, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(compacting, OperationType.UNKNOWN);
+             SSTableRewriter rewriter = new SSTableRewriter(cfs, txn, 1000, false);
+        )
         {
             rewriter.switchWriter(getWriter(cfs, s.descriptor.directory));
             while (scanner.hasNext())
@@ -746,15 +748,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 keyCount++;
                 validateKeys(keyspace);
             }
-            try
-            {
-                cfs.getDataTracker().markCompactedSSTablesReplaced(compacting, rewriter.finish(), OperationType.COMPACTION);
-                cfs.getDataTracker().unmarkCompacting(compacting);
-            }
-            catch (Throwable t)
-            {
-                rewriter.abort();
-            }
+            rewriter.finish();
         }
         validateKeys(keyspace);
         SSTableDeletingTask.waitForDeletions();
@@ -762,7 +756,7 @@ public class SSTableRewriterTest extends SchemaLoader
     }
 
     @Test
-    public void testCanonicalView() throws IOException
+    public void testCanonicalView() throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
@@ -770,15 +764,16 @@ public class SSTableRewriterTest extends SchemaLoader
 
         SSTableReader s = writeFile(cfs, 1000);
         cfs.addSSTable(s);
-        Set<SSTableReader> sstables = Sets.newHashSet(cfs.markAllCompacting());
+        Set<SSTableReader> sstables = Sets.newHashSet(s);
         assertEquals(1, sstables.size());
         SSTableRewriter.overrideOpenInterval(10000000);
-        SSTableRewriter writer = new SSTableRewriter(cfs, sstables, 1000, false);
         boolean checked = false;
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+        try (ISSTableScanner scanner = sstables.iterator().next().getScanner();
+             CompactionController controller = new CompactionController(cfs, sstables, 0);
+             LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN);
+             SSTableRewriter writer = new SSTableRewriter(cfs, txn, 1000, false);
+        )
         {
-            ISSTableScanner scanner = scanners.scanners.get(0);
-            CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
             writer.switchWriter(getWriter(cfs, sstables.iterator().next().descriptor.directory));
             while (scanner.hasNext())
             {
@@ -796,10 +791,7 @@ public class SSTableRewriterTest extends SchemaLoader
                 }
             }
         }
-        writer.abort();
-        cfs.getDataTracker().unmarkCompacting(sstables);
-        cfs.truncateBlocking();
-        SSTableDeletingTask.waitForDeletions();
+        truncateCF();
         validateCFS(cfs);
     }
 
@@ -813,30 +805,52 @@ public class SSTableRewriterTest extends SchemaLoader
         }
     }
 
-    private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+    public static void truncate(ColumnFamilyStore cfs)
     {
-        ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        for (int i = 0; i < count / 100; i++)
-            cf.addColumn(Util.cellname(i), random(0, 1000), 1);
-        File dir = cfs.directories.getDirectoryForNewSSTables();
-        String filename = cfs.getTempSSTablePath(dir);
+        cfs.truncateBlocking();
+        Uninterruptibles.sleepUninterruptibly(10L,TimeUnit.MILLISECONDS);
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
+    }
 
-        try (SSTableWriter writer = SSTableWriter.create(filename, 0, 0);)
+    public static SSTableReader writeFile(ColumnFamilyStore cfs, int count)
+    {
+        return Iterables.getFirst(writeFiles(cfs, 1, count * 5, count / 100, 1000), null);
+    }
+
+    public static Set<SSTableReader> writeFiles(ColumnFamilyStore cfs, int fileCount, int partitionCount, int cellCount, int cellSize)
+    {
+        int i = 0;
+        Set<SSTableReader> result = new LinkedHashSet<>();
+        for (int f = 0 ; f < fileCount ; f++)
         {
-            for (int i = 0; i < count * 5; i++)
+            File dir = cfs.directories.getDirectoryForNewSSTables();
+            String filename = cfs.getTempSSTablePath(dir);
+
+            SSTableWriter writer = SSTableWriter.create(filename, 0, 0);
+            int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
+            for ( ; i < end ; i++)
+            {
+                ArrayBackedSortedColumns cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
+                for (int j = 0; j < cellCount ; j++)
+                    cf.addColumn(Util.cellname(j), random(0, cellSize), 1);
                 writer.append(StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(i)), cf);
-            return writer.finish(true);
+            }
+            result.add(writer.finish(true));
         }
+        return result;
     }
 
-    private void validateCFS(ColumnFamilyStore cfs)
+    public static void validateCFS(ColumnFamilyStore cfs)
     {
         Set<Integer> liveDescriptors = new HashSet<>();
+        long spaceUsed = 0;
         for (SSTableReader sstable : cfs.getSSTables())
         {
             assertFalse(sstable.isMarkedCompacted());
             assertEquals(1, sstable.selfRef().globalCount());
             liveDescriptors.add(sstable.descriptor.generation);
+            spaceUsed += sstable.bytesOnDisk();
         }
         for (File dir : cfs.directories.getCFDirectories())
         {
@@ -849,11 +863,13 @@ public class SSTableRewriterTest extends SchemaLoader
                 }
             }
         }
-        assertTrue(cfs.getDataTracker().getCompacting().isEmpty());
+        assertEquals(spaceUsed, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(spaceUsed, cfs.metric.totalDiskSpaceUsed.getCount());
+        assertTrue(cfs.getTracker().getCompacting().isEmpty());
     }
 
 
-    private int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
+    public static int assertFileCounts(String [] files, int expectedtmplinkCount, int expectedtmpCount)
     {
         int tmplinkcount = 0;
         int tmpcount = 0;
@@ -874,7 +890,7 @@ public class SSTableRewriterTest extends SchemaLoader
         return datacount;
     }
 
-    private SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
+    public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory)
     {
         String filename = cfs.getTempSSTablePath(directory);
         return SSTableWriter.create(filename, 0, 0);


[7/7] cassandra git commit: Merge branch 'cassandra-2.2' into trunk

Posted by be...@apache.org.
Merge branch 'cassandra-2.2' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d96a02a1
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d96a02a1
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d96a02a1

Branch: refs/heads/trunk
Commit: d96a02a12e20bc2e0b096904ebe7a1d68532d2f3
Parents: 15d424e e5a76bd
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Fri May 22 09:44:46 2015 +0100
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Fri May 22 09:44:46 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 249 ++++--
 .../org/apache/cassandra/db/DataTracker.java    | 793 -------------------
 .../cassandra/db/HintedHandOffManager.java      |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |  15 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  22 +-
 .../compaction/AbstractCompactionStrategy.java  |   7 +-
 .../db/compaction/AbstractCompactionTask.java   |  19 +-
 .../db/compaction/CompactionController.java     |   4 +-
 .../db/compaction/CompactionManager.java        | 182 +++--
 .../cassandra/db/compaction/CompactionTask.java |  54 +-
 .../DateTieredCompactionStrategy.java           |  17 +-
 .../compaction/LeveledCompactionStrategy.java   |  23 +-
 .../db/compaction/LeveledCompactionTask.java    |  11 +-
 .../db/compaction/LeveledManifest.java          |  11 +-
 .../db/compaction/SSTableSplitter.java          |  13 +-
 .../cassandra/db/compaction/Scrubber.java       |  21 +-
 .../SizeTieredCompactionStrategy.java           |  30 +-
 .../cassandra/db/compaction/Upgrader.java       |  15 +-
 .../compaction/WrappingCompactionStrategy.java  |   2 +-
 .../writers/CompactionAwareWriter.java          |  11 +-
 .../writers/DefaultCompactionWriter.java        |  11 +-
 .../writers/MajorLeveledCompactionWriter.java   |  11 +-
 .../writers/MaxSSTableSizeWriter.java           |  10 +-
 .../SplittingSizeTieredCompactionWriter.java    |  14 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |   4 +-
 .../db/index/SecondaryIndexManager.java         |   2 +-
 .../apache/cassandra/db/lifecycle/Helpers.java  | 241 ++++++
 .../db/lifecycle/LifecycleTransaction.java      | 511 ++++++++++++
 .../db/lifecycle/SSTableIntervalTree.java       |  40 +
 .../apache/cassandra/db/lifecycle/Tracker.java  | 468 +++++++++++
 .../org/apache/cassandra/db/lifecycle/View.java | 252 ++++++
 .../io/compress/CompressionMetadata.java        |   2 +-
 .../io/sstable/IndexSummaryManager.java         | 106 ++-
 .../io/sstable/SSTableDeletingTask.java         |  27 +-
 .../cassandra/io/sstable/SSTableRewriter.java   | 295 ++-----
 .../io/sstable/format/SSTableReader.java        | 100 ++-
 .../io/sstable/format/big/BigTableWriter.java   |   6 +-
 .../cassandra/io/util/SequentialWriter.java     |   2 +-
 .../cassandra/metrics/ColumnFamilyMetrics.java  |  18 +-
 .../cassandra/streaming/StreamSession.java      |   7 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../cassandra/tools/StandaloneSplitter.java     |   7 +-
 .../cassandra/tools/StandaloneUpgrader.java     |   6 +-
 .../cassandra/utils/concurrent/Blocker.java     |  63 ++
 .../utils/concurrent/Transactional.java         |  31 +-
 .../db/compaction/LongCompactionsTest.java      |  10 +-
 test/unit/org/apache/cassandra/MockSchema.java  | 167 ++++
 test/unit/org/apache/cassandra/Util.java        |  27 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |   3 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |  58 +-
 .../db/compaction/AntiCompactionTest.java       |  51 +-
 .../compaction/CompactionAwareWriterTest.java   |  45 +-
 .../DateTieredCompactionStrategyTest.java       |   6 +-
 .../cassandra/db/lifecycle/HelpersTest.java     | 158 ++++
 .../db/lifecycle/LifecycleTransactionTest.java  | 412 ++++++++++
 .../cassandra/db/lifecycle/TrackerTest.java     | 342 ++++++++
 .../apache/cassandra/db/lifecycle/ViewTest.java | 202 +++++
 .../io/sstable/IndexSummaryManagerTest.java     | 123 ++-
 .../cassandra/io/sstable/SSTableReaderTest.java |  11 +-
 .../io/sstable/SSTableRewriterTest.java         | 250 +++---
 61 files changed, 3902 insertions(+), 1711 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d96a02a1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0d95a3b,ca87385..523ec84
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 +3.0:
 + * Decommissioned nodes will not rejoin the cluster (CASSANDRA-8801)
 + * Change gossip stabilization to use endpoit size (CASSANDRA-9401)
 +
 +
  2.2
+  * Extend Transactional API to sstable lifecycle management (CASSANDRA-8568)
   * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
   * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
   * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)


[2/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
new file mode 100644
index 0000000..d53a830
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/HelpersTest.java
@@ -0,0 +1,158 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.Map;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.big.BigTableReader;
+import org.apache.cassandra.utils.concurrent.Refs;
+
+public class HelpersTest
+{
+
+    static Set<Integer> a = set(1, 2, 3);
+    static Set<Integer> b = set(4, 5, 6);
+    static Set<Integer> c = set(7, 8, 9);
+    static Set<Integer> abc = set(1, 2, 3, 4, 5, 6, 7, 8, 9);
+
+    // this also tests orIn
+    @Test
+    public void testFilterIn()
+    {
+        check(Helpers.filterIn(abc, a), a);
+        check(Helpers.filterIn(abc, a, c), set(1, 2, 3, 7, 8, 9));
+        check(Helpers.filterIn(a, c), set());
+    }
+
+    // this also tests notIn
+    @Test
+    public void testFilterOut()
+    {
+        check(Helpers.filterOut(abc, a), set(4, 5, 6, 7, 8, 9));
+        check(Helpers.filterOut(abc, b), set(1, 2, 3, 7, 8, 9));
+        check(Helpers.filterOut(a, a), set());
+    }
+
+    @Test
+    public void testConcatUniq()
+    {
+        check(Helpers.concatUniq(a, b, a, c, b, a), abc);
+    }
+
+    @Test
+    public void testIdentityMap()
+    {
+        Integer one = new Integer(1);
+        Integer two = new Integer(2);
+        Integer three = new Integer(3);
+        Map<Integer, Integer> identity = Helpers.identityMap(set(one, two, three));
+        Assert.assertEquals(3, identity.size());
+        Assert.assertSame(one, identity.get(1));
+        Assert.assertSame(two, identity.get(2));
+        Assert.assertSame(three, identity.get(3));
+    }
+
+    @Test
+    public void testReplace()
+    {
+        boolean failure;
+        failure = false;
+        try
+        {
+            Helpers.replace(abc, a, c);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        failure = false;
+        try
+        {
+            Helpers.replace(a, abc, c);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        failure = false;
+        try
+        {
+            Map<Integer, Integer> notIdentity = ImmutableMap.of(1, new Integer(1), 2, 2, 3, 3);
+            Helpers.replace(notIdentity, a, b);
+        }
+        catch (AssertionError e)
+        {
+            failure = true;
+        }
+        Assert.assertTrue(failure);
+
+        // check it actually works when correct values provided
+        check(Helpers.replace(a, a, b), b);
+    }
+
+    private static Set<Integer> set(Integer ... contents)
+    {
+        return ImmutableSet.copyOf(contents);
+    }
+
+    private static void check(Iterable<Integer> check, Set<Integer> expected)
+    {
+        Assert.assertEquals(expected, ImmutableSet.copyOf(check));
+    }
+
+    @Test
+    public void testSetupDeletionNotification()
+    {
+        Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1), MockSchema.sstable(2));
+        Throwable accumulate = Helpers.setReplaced(readers, null);
+        Assert.assertNull(accumulate);
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isReplaced());
+        accumulate = Helpers.setReplaced(readers, null);
+        Assert.assertNotNull(accumulate);
+    }
+
+    @Test
+    public void testMarkObsolete()
+    {
+        Iterable<SSTableReader> readers = Lists.newArrayList(MockSchema.sstable(1), MockSchema.sstable(2));
+        Throwable accumulate = Helpers.markObsolete(readers, null);
+        Assert.assertNull(accumulate);
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isMarkedCompacted());
+        accumulate = Helpers.markObsolete(readers, null);
+        Assert.assertNotNull(accumulate);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
new file mode 100644
index 0000000..3153ef1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/LifecycleTransactionTest.java
@@ -0,0 +1,412 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction.ReaderState.Action;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.AbstractTransactionalTest;
+import org.apache.cassandra.utils.concurrent.Transactional.AbstractTransactional.State;
+
+import static com.google.common.base.Predicates.in;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.size;
+import static org.apache.cassandra.db.lifecycle.Helpers.idIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.orIn;
+import static org.apache.cassandra.db.lifecycle.Helpers.select;
+
+public class LifecycleTransactionTest extends AbstractTransactionalTest
+{
+    private boolean incrementalBackups;
+
+    @Before
+    public void disableIncrementalBackup()
+    {
+        incrementalBackups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+    }
+    @After
+    public void restoreIncrementalBackup()
+    {
+        DatabaseDescriptor.setIncrementalBackupsEnabled(incrementalBackups);
+    }
+
+    @Test
+    public void testUpdates() // (including obsoletion)
+    {
+        Tracker tracker = new Tracker(null, false);
+        SSTableReader[] readers = readersArray(0, 3);
+        SSTableReader[] readers2 = readersArray(0, 4);
+        SSTableReader[] readers3 = readersArray(0, 4);
+        tracker.addInitialSSTables(copyOf(readers));
+        LifecycleTransaction txn = tracker.tryModify(copyOf(readers), OperationType.UNKNOWN);
+
+        txn.update(readers2[0], true);
+        txn.obsolete(readers[1]);
+
+        Assert.assertTrue(txn.isObsolete(readers[1]));
+        Assert.assertFalse(txn.isObsolete(readers[0]));
+
+        testBadUpdate(txn, readers2[0], true);  // same reader && instances
+        testBadUpdate(txn, readers2[1], true);  // staged obsolete; cannot update
+        testBadUpdate(txn, readers3[0], true);  // same reader, diff instances
+        testBadUpdate(txn, readers2[2], false); // incorrectly declared original status
+        testBadUpdate(txn, readers2[3], true); // incorrectly declared original status
+
+        testBadObsolete(txn, readers[1]);  // staged obsolete; cannot obsolete again
+        testBadObsolete(txn, readers2[0]);  // staged update; cannot obsolete
+
+        txn.update(readers2[3], false);
+
+        Assert.assertEquals(3, tracker.getView().compacting.size());
+        txn.checkpoint();
+        Assert.assertTrue(txn.isObsolete(readers[1]));
+        Assert.assertFalse(txn.isObsolete(readers[0]));
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+        Assert.assertEquals(3, tracker.getView().sstables.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(of(readers2[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+        Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+        testBadObsolete(txn, readers[1]);  // logged obsolete; cannot obsolete again
+        testBadObsolete(txn, readers2[2]);  // never seen instance; cannot obsolete
+        testBadObsolete(txn, readers2[3]);  // non-original; cannot obsolete
+        testBadUpdate(txn, readers3[1], true);  // logged obsolete; cannot update
+        testBadUpdate(txn, readers2[0], true);  // same instance as logged update
+
+        txn.update(readers3[0], true);  // same reader as logged update, different instance
+        txn.checkpoint();
+
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+        Assert.assertEquals(3, tracker.getView().sstables.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(of(readers3[0], readers[2], readers2[3]), idIn(tracker.getView().sstablesMap)));
+        Assert.assertTrue(all(txn.current(), idIn(tracker.getView().sstablesMap)));
+
+        testBadObsolete(txn, readers2[0]); // not current version of sstable
+
+        txn.obsoleteOriginals();
+        txn.checkpoint();
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        txn.obsoleteOriginals(); // should be no-op
+        txn.checkpoint();
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        Assert.assertEquals(4, tracker.getView().compacting.size());
+    }
+
+    @Test
+    public void testCancellation()
+    {
+        Tracker tracker = new Tracker(null, false);
+        List<SSTableReader> readers = readers(0, 3);
+        tracker.addInitialSSTables(readers);
+        LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+
+        SSTableReader cancel = readers.get(0);
+        SSTableReader update = readers(1, 2).get(0);
+        SSTableReader fresh = readers(3, 4).get(0);
+        SSTableReader notPresent = readers(4, 5).get(0);
+
+        txn.cancel(cancel);
+        txn.update(update, true);
+        txn.update(fresh, false);
+
+        testBadCancel(txn, cancel);
+        testBadCancel(txn, update);
+        testBadCancel(txn, fresh);
+        testBadCancel(txn, notPresent);
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertEquals(2, tracker.getView().compacting.size());
+        Assert.assertTrue(all(readers.subList(1, 3), idIn(tracker.getView().compacting)));
+
+        txn.checkpoint();
+
+        testBadCancel(txn, cancel);
+        testBadCancel(txn, update);
+        testBadCancel(txn, fresh);
+        testBadCancel(txn, notPresent);
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertEquals(3, tracker.getView().compacting.size());
+        Assert.assertEquals(3, size(txn.current()));
+        Assert.assertTrue(all(concat(readers.subList(1, 3), of(fresh)), idIn(tracker.getView().compacting)));
+
+        txn.cancel(readers.get(2));
+        Assert.assertEquals(1, txn.originals().size());
+        Assert.assertEquals(2, tracker.getView().compacting.size());
+        Assert.assertEquals(2, size(txn.current()));
+        Assert.assertTrue(all(of(readers.get(1), fresh), idIn(tracker.getView().compacting)));
+    }
+
+    @Test
+    public void testSplit()
+    {
+        Tracker tracker = new Tracker(null, false);
+        List<SSTableReader> readers = readers(0, 3);
+        tracker.addInitialSSTables(readers);
+        LifecycleTransaction txn = tracker.tryModify(readers, OperationType.UNKNOWN);
+        LifecycleTransaction txn2 = txn.split(readers.subList(0, 1));
+        Assert.assertEquals(2, txn.originals().size());
+        Assert.assertTrue(all(readers.subList(1, 3), in(txn.originals())));
+        Assert.assertEquals(1, txn2.originals().size());
+        Assert.assertTrue(all(readers.subList(0, 1), in(txn2.originals())));
+        txn.update(readers(1, 2).get(0), true);
+        boolean failed = false;
+        try
+        {
+            txn.split(readers.subList(2, 3));
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadUpdate(LifecycleTransaction txn, SSTableReader update, boolean original)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.update(update, original);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadObsolete(LifecycleTransaction txn, SSTableReader update)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.obsolete(update);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    private static void testBadCancel(LifecycleTransaction txn, SSTableReader cancel)
+    {
+        boolean failed = false;
+        try
+        {
+            txn.cancel(cancel);
+        }
+        catch (Throwable t)
+        {
+            failed = true;
+        }
+        Assert.assertTrue(failed);
+    }
+
+    protected TestableTransaction newTest()
+    {
+        SSTableDeletingTask.waitForDeletions();
+        SSTableReader.resetTidying();
+        return new TxnTest();
+    }
+
+    private static final class TxnTest extends TestableTransaction
+    {
+        final List<SSTableReader> originals;
+        final List<SSTableReader> untouchedOriginals;
+        final List<SSTableReader> loggedUpdate;
+        final List<SSTableReader> loggedObsolete;
+        final List<SSTableReader> stagedObsolete;
+        final List<SSTableReader> loggedNew;
+        final List<SSTableReader> stagedNew;
+        final Tracker tracker;
+        final LifecycleTransaction txn;
+
+        private static Tracker tracker(List<SSTableReader> readers)
+        {
+            Tracker tracker = new Tracker(MockSchema.cfs, false);
+            tracker.addInitialSSTables(readers);
+            return tracker;
+        }
+
+        private TxnTest()
+        {
+            this(readers(0, 8));
+        }
+
+        private TxnTest(List<SSTableReader> readers)
+        {
+            this(tracker(readers), readers);
+        }
+
+        private TxnTest(Tracker tracker, List<SSTableReader> readers)
+        {
+            this(tracker, readers, tracker.tryModify(readers, OperationType.UNKNOWN));
+        }
+
+        private TxnTest(Tracker tracker, List<SSTableReader> readers, LifecycleTransaction txn)
+        {
+            super(txn);
+            this.tracker = tracker;
+            this.originals = readers;
+            this.txn = txn;
+            update(txn, loggedUpdate = readers(0, 2), true);
+            obsolete(txn, loggedObsolete = readers.subList(2, 4));
+            update(txn, loggedNew = readers(8, 10), false);
+            txn.checkpoint();
+            update(txn, stagedNew = readers(10, 12), false);
+            obsolete(txn, stagedObsolete = copyOf(concat(loggedUpdate, originals.subList(4, 6))));
+            untouchedOriginals = originals.subList(6, 8);
+        }
+
+        private ReaderState state(SSTableReader reader, State state)
+        {
+            SSTableReader original = select(reader, originals);
+            boolean isOriginal = original != null;
+
+            switch (state)
+            {
+                case ABORTED:
+                {
+                    return new ReaderState(Action.NONE, Action.NONE, original, original, isOriginal);
+                }
+
+                case READY_TO_COMMIT:
+                {
+                    ReaderState prev = state(reader, State.IN_PROGRESS);
+                    Action logged;
+                    SSTableReader visible;
+                    if (prev.staged == Action.NONE)
+                    {
+                        logged = prev.logged;
+                        visible = prev.currentlyVisible;
+                    }
+                    else
+                    {
+                        logged = prev.staged;
+                        visible = prev.nextVisible;
+                    }
+                    return new ReaderState(logged, Action.NONE, visible, visible, isOriginal);
+                }
+
+                case IN_PROGRESS:
+                {
+                    Action logged = Action.get(loggedUpdate.contains(reader) || loggedNew.contains(reader), loggedObsolete.contains(reader));
+                    Action staged = Action.get(stagedNew.contains(reader), stagedObsolete.contains(reader));
+                    SSTableReader currentlyVisible = ReaderState.visible(reader, in(loggedObsolete), loggedNew, loggedUpdate, originals);
+                    SSTableReader nextVisible = ReaderState.visible(reader, orIn(stagedObsolete, loggedObsolete), stagedNew, loggedNew, loggedUpdate, originals);
+                    return new ReaderState(logged, staged, currentlyVisible, nextVisible, isOriginal);
+                }
+            }
+            throw new IllegalStateException();
+        }
+
+        private List<Pair<SSTableReader, ReaderState>> states(State state)
+        {
+            List<Pair<SSTableReader, ReaderState>> result = new ArrayList<>();
+            for (SSTableReader reader : concat(originals, loggedNew, stagedNew))
+                result.add(Pair.create(reader, state(reader, state)));
+            return result;
+        }
+
+        protected void doAssert(State state)
+        {
+            for (Pair<SSTableReader, ReaderState> pair : states(state))
+            {
+                SSTableReader reader = pair.left;
+                ReaderState readerState = pair.right;
+
+                Assert.assertEquals(readerState, txn.state(reader));
+                Assert.assertEquals(readerState.currentlyVisible, tracker.getView().sstablesMap.get(reader));
+                if (readerState.currentlyVisible == null && readerState.nextVisible == null && !readerState.original)
+                    Assert.assertTrue(reader.selfRef().globalCount() == 0);
+            }
+        }
+
+        protected void assertInProgress() throws Exception
+        {
+            doAssert(State.IN_PROGRESS);
+        }
+
+        protected void assertPrepared() throws Exception
+        {
+            doAssert(State.READY_TO_COMMIT);
+        }
+
+        protected void assertAborted() throws Exception
+        {
+            doAssert(State.ABORTED);
+            Assert.assertEquals(0, tracker.getView().compacting.size());
+            Assert.assertEquals(8, tracker.getView().sstables.size());
+            for (SSTableReader reader : concat(loggedNew, stagedNew))
+                Assert.assertTrue(reader.selfRef().globalCount() == 0);
+        }
+
+        protected void assertCommitted() throws Exception
+        {
+            doAssert(State.READY_TO_COMMIT);
+            Assert.assertEquals(0, tracker.getView().compacting.size());
+            Assert.assertEquals(6, tracker.getView().sstables.size());
+            for (SSTableReader reader : concat(loggedObsolete, stagedObsolete))
+                Assert.assertTrue(reader.selfRef().globalCount() == 0);
+        }
+    }
+
+    private static SSTableReader[] readersArray(int lb, int ub)
+    {
+        return readers(lb, ub).toArray(new SSTableReader[0]);
+    }
+
+    private static List<SSTableReader> readers(int lb, int ub)
+    {
+        List<SSTableReader> readers = new ArrayList<>();
+        for (int i = lb ; i < ub ; i++)
+            readers.add(MockSchema.sstable(i, i, true));
+        return copyOf(readers);
+    }
+
+    private static void update(LifecycleTransaction txn, Iterable<SSTableReader> readers, boolean originals)
+    {
+        for (SSTableReader reader : readers)
+            txn.update(reader, originals);
+    }
+
+    private static void obsolete(LifecycleTransaction txn, Iterable<SSTableReader> readers)
+    {
+        for (SSTableReader reader : readers)
+            txn.obsolete(reader);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
new file mode 100644
index 0000000..1eef7b0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -0,0 +1,342 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import javax.annotation.Nullable;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static java.util.Collections.singleton;
+
+public class TrackerTest
+{
+
+    private static final class MockListener implements INotificationConsumer
+    {
+        final boolean throwException;
+        final List<INotification> received = new ArrayList<>();
+        final List<Object> senders = new ArrayList<>();
+
+        private MockListener(boolean throwException)
+        {
+            this.throwException = throwException;
+        }
+
+        public void handleNotification(INotification notification, Object sender)
+        {
+            if (throwException)
+                throw new RuntimeException();
+            received.add(notification);
+            senders.add(sender);
+        }
+    }
+
+    @Test
+    public void testTryModify()
+    {
+        Tracker tracker = new Tracker(MockSchema.cfs, false);
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0), MockSchema.sstable(1), MockSchema.sstable(2));
+        tracker.addInitialSSTables(copyOf(readers));
+        try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION);)
+        {
+            Assert.assertNotNull(txn);
+            Assert.assertNull(tracker.tryModify(readers.get(0), OperationType.COMPACTION));
+            Assert.assertEquals(1, txn.originals().size());
+            Assert.assertTrue(txn.originals().contains(readers.get(0)));
+        }
+        try (LifecycleTransaction txn = tracker.tryModify(Collections.<SSTableReader>emptyList(), OperationType.COMPACTION);)
+        {
+            Assert.assertNotNull(txn);
+            Assert.assertEquals(0, txn.originals().size());
+        }
+    }
+
+    @Test
+    public void testApply()
+    {
+        final Tracker tracker = new Tracker(null, false);
+        final View resultView = ViewTest.fakeView(0, 0);
+        final AtomicInteger count = new AtomicInteger();
+        tracker.apply(new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                // confound the CAS by swapping the view, and check we retry
+                if (count.incrementAndGet() < 3)
+                    tracker.view.set(ViewTest.fakeView(0, 0));
+                return true;
+            }
+        }, new Function<View, View>()
+        {
+            @Nullable
+            public View apply(View view)
+            {
+                return resultView;
+            }
+        });
+        Assert.assertEquals(3, count.get());
+        Assert.assertEquals(resultView, tracker.getView());
+
+        count.set(0);
+        // check that if the predicate returns false, we stop immediately and return null
+        Assert.assertNull(tracker.apply(new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                count.incrementAndGet();
+                return false;
+            }
+        }, null));
+        Assert.assertEquals(1, count.get());
+        Assert.assertEquals(resultView, tracker.getView());
+    }
+
+    @Test
+    public void testAddInitialSSTables()
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = new Tracker(cfs, false);
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17), MockSchema.sstable(1, 121), MockSchema.sstable(2, 9));
+        tracker.addInitialSSTables(copyOf(readers));
+
+        Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isDeleteNotificationSetup());
+
+        Assert.assertEquals(17 + 121 + 9, cfs.metric.liveDiskSpaceUsed.getCount());
+    }
+
+    @Test
+    public void testAddSSTables()
+    {
+        boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+        Tracker tracker = new Tracker(MockSchema.cfs, false);
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 17), MockSchema.sstable(1, 121), MockSchema.sstable(2, 9));
+        tracker.addSSTables(copyOf(readers));
+
+        Assert.assertEquals(3, tracker.view.get().sstables.size());
+
+        for (SSTableReader reader : readers)
+            Assert.assertTrue(reader.isDeleteNotificationSetup());
+
+        Assert.assertEquals(17 + 121 + 9, MockSchema.cfs.metric.liveDiskSpaceUsed.getCount());
+        Assert.assertEquals(3, listener.senders.size());
+        Assert.assertEquals(tracker, listener.senders.get(0));
+        Assert.assertTrue(listener.received.get(0) instanceof SSTableAddedNotification);
+        DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+    }
+
+    @Test
+    public void testDropSSTables()
+    {
+        testDropSSTables(false);
+        SSTableDeletingTask.waitForDeletions();
+        testDropSSTables(true);
+        SSTableDeletingTask.waitForDeletions();
+    }
+
+    private void testDropSSTables(boolean invalidate)
+    {
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        Tracker tracker = cfs.getTracker();
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        final List<SSTableReader> readers = ImmutableList.of(MockSchema.sstable(0, 9, true), MockSchema.sstable(1, 15, true), MockSchema.sstable(2, 71, true));
+        tracker.addInitialSSTables(copyOf(readers));
+        try (LifecycleTransaction txn = tracker.tryModify(readers.get(0), OperationType.COMPACTION);)
+        {
+            SSTableDeletingTask.pauseDeletions(true);
+            if (invalidate)
+                cfs.invalidate(false);
+            else
+                tracker.dropSSTables();
+            Assert.assertEquals(95, cfs.metric.totalDiskSpaceUsed.getCount());
+            Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+            Assert.assertEquals(1, tracker.getView().sstables.size());
+            SSTableDeletingTask.pauseDeletions(false);
+        }
+        if (!invalidate)
+        {
+            Assert.assertEquals(1, tracker.getView().sstables.size());
+            Assert.assertEquals(readers.get(0), Iterables.getFirst(tracker.getView().sstables, null));
+            Assert.assertEquals(1, readers.get(0).selfRef().globalCount());
+            Assert.assertFalse(readers.get(0).isMarkedCompacted());
+            for (SSTableReader reader : readers.subList(1, 3))
+            {
+                Assert.assertEquals(0, reader.selfRef().globalCount());
+                Assert.assertTrue(reader.isMarkedCompacted());
+            }
+            Assert.assertNull(tracker.dropSSTables(new Predicate<SSTableReader>()
+            {
+                public boolean apply(SSTableReader reader)
+                {
+                    return reader != readers.get(0);
+                }
+            }, OperationType.UNKNOWN, null));
+
+            Assert.assertEquals(1, tracker.getView().sstables.size());
+            Assert.assertEquals(1, listener.received.size());
+            Assert.assertEquals(tracker, listener.senders.get(0));
+            Assert.assertEquals(2, ((SSTableListChangedNotification) listener.received.get(0)).removed.size());
+            Assert.assertEquals(0, ((SSTableListChangedNotification) listener.received.get(0)).added.size());
+            Assert.assertEquals(9, cfs.metric.liveDiskSpaceUsed.getCount());
+            readers.get(0).selfRef().release();
+        }
+        else
+        {
+            Assert.assertEquals(0, tracker.getView().sstables.size());
+            Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+            for (SSTableReader reader : readers)
+                Assert.assertTrue(reader.isMarkedCompacted());
+        }
+    }
+
+    @Test
+    public void testMemtableReplacement()
+    {
+        boolean backups = DatabaseDescriptor.isIncrementalBackupsEnabled();
+        DatabaseDescriptor.setIncrementalBackupsEnabled(false);
+        ColumnFamilyStore cfs = MockSchema.newCFS();
+        MockListener listener = new MockListener(false);
+        Tracker tracker = cfs.getTracker();
+        tracker.subscribe(listener);
+
+        Memtable prev1 = tracker.switchMemtable(true);
+        OpOrder.Group write1 = cfs.keyspace.writeOrder.getCurrent();
+        OpOrder.Barrier barrier1 = cfs.keyspace.writeOrder.newBarrier();
+        prev1.setDiscarding(barrier1, new AtomicReference<ReplayPosition>());
+        barrier1.issue();
+        Memtable prev2 = tracker.switchMemtable(false);
+        OpOrder.Group write2 = cfs.keyspace.writeOrder.getCurrent();
+        OpOrder.Barrier barrier2 = cfs.keyspace.writeOrder.newBarrier();
+        prev2.setDiscarding(barrier2, new AtomicReference<ReplayPosition>());
+        barrier2.issue();
+        Memtable cur = tracker.getView().getCurrentMemtable();
+        OpOrder.Group writecur = cfs.keyspace.writeOrder.getCurrent();
+        Assert.assertEquals(prev1, tracker.getMemtableFor(write1, ReplayPosition.NONE));
+        Assert.assertEquals(prev2, tracker.getMemtableFor(write2, ReplayPosition.NONE));
+        Assert.assertEquals(cur, tracker.getMemtableFor(writecur, ReplayPosition.NONE));
+        Assert.assertEquals(1, listener.received.size());
+        Assert.assertTrue(listener.received.get(0) instanceof MemtableRenewedNotification);
+        listener.received.clear();
+
+        tracker.markFlushing(prev2);
+        Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+        tracker.markFlushing(prev1);
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev1));
+        Assert.assertEquals(2, tracker.getView().flushingMemtables.size());
+
+        tracker.replaceFlushed(prev1, null);
+        Assert.assertEquals(1, tracker.getView().flushingMemtables.size());
+        Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
+
+        SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
+        tracker.replaceFlushed(prev2, reader);
+        Assert.assertEquals(1, tracker.getView().sstables.size());
+        Assert.assertEquals(1, listener.received.size());
+        Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        Assert.assertTrue(reader.isDeleteNotificationSetup());
+        Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
+
+        // test invalidated CFS
+        cfs = MockSchema.newCFS();
+        tracker = cfs.getTracker();
+        listener = new MockListener(false);
+        tracker.subscribe(listener);
+        prev1 = tracker.switchMemtable(false);
+        tracker.markFlushing(prev1);
+        reader = MockSchema.sstable(0, 10, true, cfs);
+        cfs.invalidate(false);
+        tracker.replaceFlushed(prev1, reader);
+        Assert.assertEquals(0, tracker.getView().sstables.size());
+        Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
+        Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        Assert.assertEquals(reader, ((SSTableAddedNotification) listener.received.get(0)).added);
+        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+        DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
+    }
+
+    @Test
+    public void testNotifications()
+    {
+        SSTableReader r1 = MockSchema.sstable(0), r2 = MockSchema.sstable(1);
+        Tracker tracker = new Tracker(null, false);
+        MockListener listener = new MockListener(false);
+        tracker.subscribe(listener);
+        tracker.notifyAdded(r1);
+        Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        tracker.notifyDeleting(r1);
+        Assert.assertEquals(r1, ((SSTableDeletingNotification) listener.received.get(0)).deleting);
+        listener.received.clear();
+        Assert.assertNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+        Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+        Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        tracker.notifySSTableRepairedStatusChanged(singleton(r1));
+        Assert.assertEquals(singleton(r1), ((SSTableRepairStatusChanged) listener.received.get(0)).sstable);
+        listener.received.clear();
+        Memtable memtable = MockSchema.memtable();
+        tracker.notifyRenewed(memtable);
+        Assert.assertEquals(memtable, ((MemtableRenewedNotification) listener.received.get(0)).renewed);
+        listener.received.clear();
+        tracker.unsubscribe(listener);
+        MockListener failListener = new MockListener(true);
+        tracker.subscribe(failListener);
+        tracker.subscribe(listener);
+        Assert.assertNotNull(tracker.notifyAdded(r1, null));
+        Assert.assertEquals(r1, ((SSTableAddedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+        Assert.assertNotNull(tracker.notifySSTablesChanged(singleton(r1), singleton(r2), OperationType.COMPACTION, null));
+        Assert.assertEquals(singleton(r1), ((SSTableListChangedNotification) listener.received.get(0)).removed);
+        Assert.assertEquals(singleton(r2), ((SSTableListChangedNotification) listener.received.get(0)).added);
+        listener.received.clear();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
new file mode 100644
index 0000000..811e025
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -0,0 +1,202 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.google.common.base.Function;
+import com.google.common.base.Predicates;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.MockSchema;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.ImmutableSet.of;
+import static com.google.common.collect.Iterables.concat;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+
+public class ViewTest
+{
+
+    @Test
+    public void testSSTablesInBounds()
+    {
+        View initialView = fakeView(0, 5);
+        for (int i = 0 ; i < 5 ; i++)
+        {
+            for (int j = i ; j < 5 ; j++)
+            {
+                RowPosition min = MockSchema.readerBounds(i);
+                RowPosition max = MockSchema.readerBounds(j);
+                for (boolean minInc : new boolean[] { true, false} )
+                {
+                    for (boolean maxInc : new boolean[] { true, false} )
+                    {
+                        if (i == j && !(minInc && maxInc))
+                            continue;
+                        List<SSTableReader> r = initialView.sstablesInBounds(AbstractBounds.bounds(min, minInc, max, maxInc));
+                        Assert.assertEquals(String.format("%d(%s) %d(%s)", i, minInc, j, maxInc), j - i + (minInc ? 0 : -1) + (maxInc ? 1 : 0), r.size());
+                    }
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testCompaction()
+    {
+        View initialView = fakeView(0, 5);
+        View cur = initialView;
+        List<SSTableReader> readers = ImmutableList.copyOf(initialView.sstables);
+        Assert.assertTrue(View.permitCompacting(readers).apply(cur));
+        // check we permit compacting duplicates in the predicate, so we don't spin infinitely if there is a screw up
+        Assert.assertTrue(View.permitCompacting(ImmutableList.copyOf(concat(readers, readers))).apply(cur));
+        // check we fail in the application in the presence of duplicates
+        testFailure(View.updateCompacting(emptySet(), concat(readers.subList(0, 1), readers.subList(0, 1))), cur);
+
+        // do lots of trivial checks that the compacting set and related methods behave properly for a simple update
+        cur = View.updateCompacting(emptySet(), readers.subList(0, 2)).apply(cur);
+        Assert.assertTrue(View.permitCompacting(readers.subList(2, 5)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(0, 2)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(0, 1)).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+        Assert.assertTrue(readers.subList(2, 5).containsAll(copyOf(cur.getUncompacting(readers))));
+        Assert.assertEquals(3, copyOf(cur.getUncompacting(readers)).size());
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+        Assert.assertEquals(3, cur.nonCompactingSStables().size());
+
+        // check marking already compacting readers fails with an exception
+        testFailure(View.updateCompacting(emptySet(), readers.subList(0, 1)), cur);
+        testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+        testFailure(View.updateCompacting(copyOf(readers.subList(0, 1)), readers.subList(1, 2)), cur);
+
+        // make equivalents of readers.subList(0, 3) that are different instances
+        SSTableReader r0 = MockSchema.sstable(0), r1 = MockSchema.sstable(1), r2 = MockSchema.sstable(2);
+        // attempt to mark compacting a version not in the live set
+        testFailure(View.updateCompacting(emptySet(), of(r2)), cur);
+        // update one compacting, one non-compacting, of the liveset to another instance of the same readers;
+        // confirm liveset changes but compacting does not
+        cur = View.updateLiveSet(copyOf(readers.subList(1, 3)), of(r1, r2)).apply(cur);
+        Assert.assertSame(readers.get(0), cur.sstablesMap.get(r0));
+        Assert.assertSame(r1, cur.sstablesMap.get(r1));
+        Assert.assertSame(r2, cur.sstablesMap.get(r2));
+        testFailure(View.updateCompacting(emptySet(), readers.subList(2, 3)), cur);
+        Assert.assertSame(readers.get(1), Iterables.getFirst(Iterables.filter(cur.compacting, Predicates.equalTo(r1)), null));
+
+        // unmark compacting, and check our methods are all correctly updated
+        cur = View.updateCompacting(copyOf(readers.subList(0, 1)), emptySet()).apply(cur);
+        Assert.assertTrue(View.permitCompacting(concat(readers.subList(0, 1), of(r2), readers.subList(3, 5))).apply(cur));
+        Assert.assertFalse(View.permitCompacting(readers.subList(1, 2)).apply(cur));
+        testFailure(View.updateCompacting(emptySet(), readers.subList(1, 2)), cur);
+        testFailure(View.updateCompacting(copyOf(readers.subList(0, 2)), emptySet()), cur);
+        Assert.assertTrue(copyOf(concat(readers.subList(0, 1), readers.subList(2, 5))).containsAll(copyOf(cur.getUncompacting(readers))));
+        Assert.assertEquals(4, copyOf(cur.getUncompacting(readers)).size());
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(2, 5)));
+        Assert.assertTrue(cur.nonCompactingSStables().containsAll(readers.subList(0, 1)));
+        Assert.assertEquals(4, cur.nonCompactingSStables().size());
+    }
+
+    private static void testFailure(Function<View, ?> function, View view)
+    {
+        boolean failed = true;
+        try
+        {
+            function.apply(view);
+            failed = false;
+        }
+        catch (Throwable t)
+        {
+        }
+        Assert.assertTrue(failed);
+    }
+
+    @Test
+    public void testFlushing()
+    {
+        View initialView = fakeView(1, 0);
+        View cur = initialView;
+        Memtable memtable1 = initialView.getCurrentMemtable();
+        Memtable memtable2 = MockSchema.memtable();
+
+        cur = View.switchMemtable(memtable2).apply(cur);
+        Assert.assertEquals(2, cur.liveMemtables.size());
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.getCurrentMemtable());
+
+        Memtable memtable3 = MockSchema.memtable();
+        cur = View.switchMemtable(memtable3).apply(cur);
+        Assert.assertEquals(3, cur.liveMemtables.size());
+        Assert.assertEquals(0, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.liveMemtables.get(1));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        testFailure(View.replaceFlushed(memtable2, null), cur);
+
+        cur = View.markFlushing(memtable2).apply(cur);
+        Assert.assertTrue(cur.flushingMemtables.contains(memtable2));
+        Assert.assertEquals(2, cur.liveMemtables.size());
+        Assert.assertEquals(1, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable2, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable1, cur.liveMemtables.get(0));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        cur = View.markFlushing(memtable1).apply(cur);
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(2, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable2, cur.flushingMemtables.get(1));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        cur = View.replaceFlushed(memtable2, null).apply(cur);
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(1, cur.flushingMemtables.size());
+        Assert.assertEquals(memtable1, cur.flushingMemtables.get(0));
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+
+        SSTableReader sstable = MockSchema.sstable(1);
+        cur = View.replaceFlushed(memtable1, sstable).apply(cur);
+        Assert.assertEquals(0, cur.flushingMemtables.size());
+        Assert.assertEquals(1, cur.liveMemtables.size());
+        Assert.assertEquals(memtable3, cur.getCurrentMemtable());
+        Assert.assertEquals(1, cur.sstables.size());
+        Assert.assertEquals(sstable, cur.sstablesMap.get(sstable));
+    }
+
+    static View fakeView(int memtableCount, int sstableCount)
+    {
+        List<Memtable> memtables = new ArrayList<>();
+        List<SSTableReader> sstables = new ArrayList<>();
+        for (int i = 0 ; i < memtableCount ; i++)
+            memtables.add(MockSchema.memtable());
+        for (int i = 0 ; i < sstableCount ; i++)
+            sstables.add(MockSchema.sstable(i));
+        return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
+                        Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index eac6094..518d80e 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -41,13 +41,15 @@ import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.metrics.RestorableMeter;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 
+import static com.google.common.collect.ImmutableMap.of;
+import static java.util.Arrays.asList;
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
 import static org.apache.cassandra.io.sstable.IndexSummaryManager.DOWNSAMPLE_THESHOLD;
 import static org.apache.cassandra.io.sstable.IndexSummaryManager.UPSAMPLE_THRESHOLD;
@@ -121,12 +123,15 @@ public class IndexSummaryManagerTest
         return total;
     }
 
-    private static List<SSTableReader> resetSummaries(List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
+    private static List<SSTableReader> resetSummaries(ColumnFamilyStore cfs, List<SSTableReader> sstables, long originalOffHeapSize) throws IOException
     {
         for (SSTableReader sstable : sstables)
             sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
 
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, originalOffHeapSize * sstables.size());
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), originalOffHeapSize * sstables.size());
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
 
@@ -230,7 +235,11 @@ public class IndexSummaryManagerTest
         cfs.metadata.minIndexInterval(originalMinIndexInterval / 2);
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         long summarySpace = sstable.getIndexSummaryOffHeapSize();
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), summarySpace);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), summarySpace);
+        }
+
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(originalMinIndexInterval, sstable.getEffectiveIndexInterval(), 0.001);
         assertEquals(numRows / originalMinIndexInterval, sstable.getIndexSummarySize());
@@ -238,7 +247,10 @@ public class IndexSummaryManagerTest
         // keep the min_index_interval the same, but now give the summary enough space to grow by 50%
         double previousInterval = sstable.getEffectiveIndexInterval();
         int previousSize = sstable.getIndexSummarySize();
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace * 1.5));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace * 1.5));
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(previousSize * 1.5, (double) sstable.getIndexSummarySize(), 1);
         assertEquals(previousInterval * (1.0 / 1.5), sstable.getEffectiveIndexInterval(), 0.001);
@@ -246,7 +258,10 @@ public class IndexSummaryManagerTest
         // return min_index_interval to it's original value (double it), but only give the summary enough space
         // to have an effective index interval of twice the new min
         cfs.metadata.minIndexInterval(originalMinIndexInterval);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), (long) Math.ceil(summarySpace / 2.0));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) Math.ceil(summarySpace / 2.0));
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(originalMinIndexInterval * 2, sstable.getEffectiveIndexInterval(), 0.001);
         assertEquals(numRows / (originalMinIndexInterval * 2), sstable.getIndexSummarySize());
@@ -256,7 +271,10 @@ public class IndexSummaryManagerTest
         // result in an effective interval above the new max)
         cfs.metadata.minIndexInterval(originalMinIndexInterval * 4);
         cfs.metadata.maxIndexInterval(originalMinIndexInterval * 4);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, Arrays.asList(sstable), 10);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         sstable = cfs.getSSTables().iterator().next();
         assertEquals(cfs.metadata.getMinIndexInterval(), sstable.getEffectiveIndexInterval(), 0.001);
     }
@@ -276,14 +294,20 @@ public class IndexSummaryManagerTest
         for (SSTableReader sstable : sstables)
             sstable.overrideReadMeter(new RestorableMeter(100.0, 100.0));
 
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         sstables = new ArrayList<>(cfs.getSSTables());
         for (SSTableReader sstable : sstables)
             assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
 
         // halve the max_index_interval
         cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() / 2);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+        }
         sstables = new ArrayList<>(cfs.getSSTables());
         for (SSTableReader sstable : sstables)
         {
@@ -293,7 +317,10 @@ public class IndexSummaryManagerTest
 
         // return max_index_interval to its original value
         cfs.metadata.maxIndexInterval(cfs.metadata.getMaxIndexInterval() * 2);
-        IndexSummaryManager.redistributeSummaries(Collections.EMPTY_LIST, sstables, 1);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 1);
+        }
         for (SSTableReader sstable : cfs.getSSTables())
         {
             assertEquals(cfs.metadata.getMaxIndexInterval(), sstable.getEffectiveIndexInterval(), 0.01);
@@ -321,7 +348,10 @@ public class IndexSummaryManagerTest
         long singleSummaryOffHeapSpace = sstables.get(0).getIndexSummaryOffHeapSize();
 
         // there should be enough space to not downsample anything
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
         assertEquals(singleSummaryOffHeapSpace * numSSTables, totalOffHeapSize(sstables));
@@ -329,26 +359,38 @@ public class IndexSummaryManagerTest
 
         // everything should get cut in half
         assert sstables.size() == 4;
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 2)));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2)));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // everything should get cut to a quarter
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * (numSSTables / 4)));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 4)));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 4, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // upsample back up to half
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables,(singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * (numSSTables / 2) + 4));
+        }
         assert sstables.size() == 4;
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL / 2, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
 
         // upsample back up to the original index summary
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * numSSTables));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * numSSTables));
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(BASE_SAMPLING_LEVEL, sstable.getIndexSummarySamplingLevel());
         validateData(cfs, numRows);
@@ -357,7 +399,10 @@ public class IndexSummaryManagerTest
         // so the two cold sstables should get downsampled to be half of their original size
         sstables.get(0).overrideReadMeter(new RestorableMeter(50.0, 50.0));
         sstables.get(1).overrideReadMeter(new RestorableMeter(50.0, 50.0));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -370,7 +415,10 @@ public class IndexSummaryManagerTest
         double higherRate = 50.0 * (UPSAMPLE_THRESHOLD - (UPSAMPLE_THRESHOLD * 0.10));
         sstables.get(0).overrideReadMeter(new RestorableMeter(lowerRate, lowerRate));
         sstables.get(1).overrideReadMeter(new RestorableMeter(higherRate, higherRate));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(0).getIndexSummarySamplingLevel());
         assertEquals(BASE_SAMPLING_LEVEL / 2, sstables.get(1).getIndexSummarySamplingLevel());
@@ -379,13 +427,16 @@ public class IndexSummaryManagerTest
         validateData(cfs, numRows);
 
         // reset, and then this time, leave enough space for one of the cold sstables to not get downsampled
-        sstables = resetSummaries(sstables, singleSummaryOffHeapSpace);
+        sstables = resetSummaries(cfs, sstables, singleSummaryOffHeapSpace);
         sstables.get(0).overrideReadMeter(new RestorableMeter(1.0, 1.0));
         sstables.get(1).overrideReadMeter(new RestorableMeter(2.0, 2.0));
         sstables.get(2).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
         sstables.get(3).overrideReadMeter(new RestorableMeter(1000.0, 1000.0));
 
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (singleSummaryOffHeapSpace * 3) + 50);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (singleSummaryOffHeapSpace * 3) + 50);
+        }
         Collections.sort(sstables, hotnessComparator);
 
         if (sstables.get(0).getIndexSummarySamplingLevel() == minSamplingLevel)
@@ -406,7 +457,10 @@ public class IndexSummaryManagerTest
         sstables.get(1).overrideReadMeter(new RestorableMeter(0.0, 0.0));
         sstables.get(2).overrideReadMeter(new RestorableMeter(92, 92));
         sstables.get(3).overrideReadMeter(new RestorableMeter(128.0, 128.0));
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), (long) (singleSummaryOffHeapSpace + (singleSummaryOffHeapSpace * (92.0 / BASE_SAMPLING_LEVEL))));
+        }
         Collections.sort(sstables, hotnessComparator);
         assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
         assertEquals(1, sstables.get(0).getIndexSummarySize());  // at the min sampling level
@@ -416,7 +470,10 @@ public class IndexSummaryManagerTest
         validateData(cfs, numRows);
 
         // Don't leave enough space for even the minimal index summaries
-        sstables = redistributeSummaries(Collections.EMPTY_LIST, sstables, 10);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.UNKNOWN))
+        {
+            sstables = redistributeSummaries(Collections.EMPTY_LIST, of(cfs.metadata.cfId, txn), 10);
+        }
         for (SSTableReader sstable : sstables)
             assertEquals(1, sstable.getIndexSummarySize());  // at the min sampling level
         validateData(cfs, numRows);
@@ -449,19 +506,19 @@ public class IndexSummaryManagerTest
         SSTableReader original = sstables.get(0);
 
         SSTableReader sstable = original;
-        for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(asList(sstable), OperationType.UNKNOWN))
         {
-            SSTableReader prev = sstable;
-            sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
-            assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
-            int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
-            assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
-            if (prev != original)
-                prev.selfRef().release();
+            for (int samplingLevel = 1; samplingLevel < BASE_SAMPLING_LEVEL; samplingLevel++)
+            {
+                sstable = sstable.cloneWithNewSummarySamplingLevel(cfs, samplingLevel);
+                assertEquals(samplingLevel, sstable.getIndexSummarySamplingLevel());
+                int expectedSize = (numRows * samplingLevel) / (sstable.metadata.getMinIndexInterval() * BASE_SAMPLING_LEVEL);
+                assertEquals(expectedSize, sstable.getIndexSummarySize(), 1);
+                txn.update(sstable, true);
+                txn.checkpoint();
+            }
+            txn.finish();
         }
-
-        // don't leave replaced SSTRs around to break other tests
-        cfs.getDataTracker().replaceWithNewInstances(Collections.singleton(original), Collections.singleton(sstable));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 82cb8d5..682d999 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -59,7 +59,9 @@ import org.apache.cassandra.db.Row;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.composites.Composites;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.LocalPartitioner;
 import org.apache.cassandra.dht.LocalPartitioner.LocalToken;
 import org.apache.cassandra.dht.Range;
@@ -498,8 +500,13 @@ public class SSTableReaderTest
             }));
         }
 
-        SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
-        store.getDataTracker().replaceWithNewInstances(Arrays.asList(sstable), Arrays.asList(replacement));
+        SSTableReader replacement;
+        try (LifecycleTransaction txn = store.getTracker().tryModify(Arrays.asList(sstable), OperationType.UNKNOWN))
+        {
+            replacement = sstable.cloneWithNewSummarySamplingLevel(store, 1);
+            txn.update(replacement, true);
+            txn.finish();
+        }
         for (Future future : futures)
             future.get();
 


[5/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index d79b835..004e893 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -54,14 +54,10 @@ import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.OnDiskAtom;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.CompactionInfo.Holder;
 import org.apache.cassandra.db.index.SecondaryIndexBuilder;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -82,12 +78,14 @@ import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.Refs;
 
+import static java.util.Collections.singleton;
+
 /**
  * <p>
  * A singleton which manages a private executor of ongoing compactions.
  * </p>
  * Scheduling for compaction is accomplished by swapping sstables to be compacted into
- * a set via DataTracker. New scheduling attempts will ignore currently compacting
+ * a set via Tracker. New scheduling attempts will ignore currently compacting
  * sstables.
  */
 public class CompactionManager implements CompactionManagerMBean
@@ -195,7 +193,7 @@ public class CompactionManager implements CompactionManagerMBean
     public boolean isCompacting(Iterable<ColumnFamilyStore> cfses)
     {
         for (ColumnFamilyStore cfs : cfses)
-            if (!cfs.getDataTracker().getCompacting().isEmpty())
+            if (!cfs.getTracker().getCompacting().isEmpty())
                 return true;
         return false;
     }
@@ -245,22 +243,22 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation) throws ExecutionException, InterruptedException
+    private AllSSTableOpStatus parallelAllSSTableOperation(final ColumnFamilyStore cfs, final OneSSTableOperation operation, OperationType operationType) throws ExecutionException, InterruptedException
     {
-        Iterable<SSTableReader> compactingSSTables = cfs.markAllCompacting();
-        if (compactingSSTables == null)
-        {
-            logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
-            return AllSSTableOpStatus.ABORTED;
-        }
-        if (Iterables.isEmpty(compactingSSTables))
+        try (LifecycleTransaction compacting = cfs.markAllCompacting(operationType);)
         {
-            logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
-            return AllSSTableOpStatus.SUCCESSFUL;
-        }
-        try
-        {
-            Iterable<SSTableReader> sstables = operation.filterSSTables(compactingSSTables);
+            if (compacting == null)
+            {
+                logger.info("Aborting operation on {}.{} after failing to interrupt other compaction operations", cfs.keyspace.getName(), cfs.name);
+                return AllSSTableOpStatus.ABORTED;
+            }
+            if (compacting.originals().isEmpty())
+            {
+                logger.info("No sstables for {}.{}", cfs.keyspace.getName(), cfs.name);
+                return AllSSTableOpStatus.SUCCESSFUL;
+            }
+
+            Iterable<SSTableReader> sstables = operation.filterSSTables(compacting.originals());
             List<Future<Object>> futures = new ArrayList<>();
 
             for (final SSTableReader sstable : sstables)
@@ -271,31 +269,30 @@ public class CompactionManager implements CompactionManagerMBean
                     return AllSSTableOpStatus.ABORTED;
                 }
 
+                final LifecycleTransaction txn = compacting.split(singleton(sstable));
                 futures.add(executor.submit(new Callable<Object>()
                 {
                     @Override
                     public Object call() throws Exception
                     {
-                        operation.execute(sstable);
+                        operation.execute(txn);
                         return this;
                     }
                 }));
             }
 
+            assert compacting.originals().isEmpty();
+
             for (Future<Object> f : futures)
                 f.get();
+            return AllSSTableOpStatus.SUCCESSFUL;
         }
-        finally
-        {
-            cfs.getDataTracker().unmarkCompacting(compactingSSTables);
-        }
-        return AllSSTableOpStatus.SUCCESSFUL;
     }
 
     private static interface OneSSTableOperation
     {
         Iterable<SSTableReader> filterSSTables(Iterable<SSTableReader> input);
-        void execute(SSTableReader input) throws IOException;
+        void execute(LifecycleTransaction input) throws IOException;
     }
 
     public enum AllSSTableOpStatus { ABORTED(1), SUCCESSFUL(0);
@@ -318,11 +315,11 @@ public class CompactionManager implements CompactionManagerMBean
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction input) throws IOException
             {
                 scrubOne(cfs, input, skipCorrupted, checkData);
             }
-        });
+        }, OperationType.SCRUB);
     }
 
     public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
@@ -337,11 +334,11 @@ public class CompactionManager implements CompactionManagerMBean
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction input) throws IOException
             {
-                verifyOne(cfs, input, extendedVerify);
+                verifyOne(cfs, input.onlyOne(), extendedVerify);
             }
-        });
+        }, OperationType.VERIFY);
     }
 
     public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion) throws InterruptedException, ExecutionException
@@ -362,14 +359,14 @@ public class CompactionManager implements CompactionManagerMBean
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction txn) throws IOException
             {
-                AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(Collections.singleton(input), NO_GC, Long.MAX_VALUE);
+                AbstractCompactionTask task = cfs.getCompactionStrategy().getCompactionTask(txn, NO_GC, Long.MAX_VALUE);
                 task.setUserDefined(true);
                 task.setCompactionType(OperationType.UPGRADE_SSTABLES);
                 task.execute(metrics);
             }
-        });
+        }, OperationType.UPGRADE_SSTABLES);
     }
 
     public AllSSTableOpStatus performCleanup(final ColumnFamilyStore cfStore) throws InterruptedException, ExecutionException
@@ -395,12 +392,12 @@ public class CompactionManager implements CompactionManagerMBean
             }
 
             @Override
-            public void execute(SSTableReader input) throws IOException
+            public void execute(LifecycleTransaction txn) throws IOException
             {
                 CleanupStrategy cleanupStrategy = CleanupStrategy.get(cfStore, ranges);
-                doCleanupOne(cfStore, input, cleanupStrategy, ranges, hasIndexes);
+                doCleanupOne(cfStore, txn, cleanupStrategy, ranges, hasIndexes);
             }
-        });
+        }, OperationType.CLEANUP);
     }
 
     public ListenableFuture<?> submitAntiCompaction(final ColumnFamilyStore cfs,
@@ -412,19 +409,19 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void runMayThrow() throws Exception
             {
-                boolean success = false;
-                while (!success)
+                LifecycleTransaction modifier = null;
+                while (modifier == null)
                 {
-                    for (SSTableReader compactingSSTable : cfs.getDataTracker().getCompacting())
+                    for (SSTableReader compactingSSTable : cfs.getTracker().getCompacting())
                         sstables.releaseIfHolds(compactingSSTable);
                     Set<SSTableReader> compactedSSTables = new HashSet<>();
                     for (SSTableReader sstable : sstables)
                         if (sstable.isMarkedCompacted())
                             compactedSSTables.add(sstable);
                     sstables.release(compactedSSTables);
-                    success = sstables.isEmpty() || cfs.getDataTracker().markCompacting(sstables);
+                    modifier = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
                 }
-                performAnticompaction(cfs, ranges, sstables, repairedAt);
+                performAnticompaction(cfs, ranges, sstables, modifier, repairedAt);
             }
         };
         if (executor.isShutdown())
@@ -452,6 +449,7 @@ public class CompactionManager implements CompactionManagerMBean
     public void performAnticompaction(ColumnFamilyStore cfs,
                                       Collection<Range<Token>> ranges,
                                       Refs<SSTableReader> validatedForRepair,
+                                      LifecycleTransaction txn,
                                       long repairedAt) throws InterruptedException, IOException
     {
         logger.info("Starting anticompaction for {}.{} on {}/{} sstables", cfs.keyspace.getName(), cfs.getColumnFamilyName(), validatedForRepair.size(), cfs.getSSTables().size());
@@ -490,16 +488,18 @@ public class CompactionManager implements CompactionManagerMBean
                     }
                 }
             }
-            cfs.getDataTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatuses);
-            cfs.getDataTracker().unmarkCompacting(Sets.union(nonAnticompacting, mutatedRepairStatuses));
+            cfs.getTracker().notifySSTableRepairedStatusChanged(mutatedRepairStatuses);
+            txn.cancel(Sets.union(nonAnticompacting, mutatedRepairStatuses));
             validatedForRepair.release(Sets.union(nonAnticompacting, mutatedRepairStatuses));
+            assert txn.originals().equals(sstables);
             if (!sstables.isEmpty())
-                doAntiCompaction(cfs, ranges, sstables, repairedAt);
+                doAntiCompaction(cfs, ranges, txn, repairedAt);
+            txn.finish();
         }
         finally
         {
             validatedForRepair.release();
-            cfs.getDataTracker().unmarkCompacting(sstables);
+            txn.close();
         }
 
         logger.info("Completed anticompaction successfully");
@@ -657,9 +657,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, false, checkData);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -750,15 +750,16 @@ public class CompactionManager implements CompactionManagerMBean
      *
      * @throws IOException
      */
-    private void doCleanupOne(final ColumnFamilyStore cfs, SSTableReader sstable, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
+    private void doCleanupOne(final ColumnFamilyStore cfs, LifecycleTransaction txn, CleanupStrategy cleanupStrategy, Collection<Range<Token>> ranges, boolean hasIndexes) throws IOException
     {
         assert !cfs.isIndex();
 
-        Set<SSTableReader> sstableSet = Collections.singleton(sstable);
+        SSTableReader sstable = txn.onlyOne();
 
         if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
         {
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstableSet, Collections.<SSTableReader>emptyList(), OperationType.CLEANUP);
+            txn.obsoleteOriginals();
+            txn.finish();
             return;
         }
         if (!needsCleanup(sstable, ranges))
@@ -772,13 +773,13 @@ public class CompactionManager implements CompactionManagerMBean
         long totalkeysWritten = 0;
 
         int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(),
-                                               (int) (SSTableReader.getApproximateKeyCount(sstableSet)));
+                                               (int) (SSTableReader.getApproximateKeyCount(txn.originals())));
         if (logger.isDebugEnabled())
             logger.debug("Expected bloom filter size : {}", expectedBloomFilterSize);
 
         logger.info("Cleaning up {}", sstable);
 
-        File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableSet, OperationType.CLEANUP));
+        File compactionFileLocation = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(txn.originals(), OperationType.CLEANUP));
         if (compactionFileLocation == null)
             throw new IOException("disk full");
 
@@ -786,10 +787,9 @@ public class CompactionManager implements CompactionManagerMBean
         CleanupInfo ci = new CleanupInfo(sstable, scanner);
 
         metrics.beginCompaction(ci);
-        Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
         List<SSTableReader> finished;
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, false);
-             CompactionController controller = new CompactionController(cfs, sstableSet, getDefaultGcBefore(cfs)))
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, txn, sstable.maxDataAge, false);
+             CompactionController controller = new CompactionController(cfs, txn.originals(), getDefaultGcBefore(cfs)))
         {
             writer.switchWriter(createWriter(cfs, compactionFileLocation, expectedBloomFilterSize, sstable.getSSTableMetadata().repairedAt, sstable));
 
@@ -811,7 +811,6 @@ public class CompactionManager implements CompactionManagerMBean
             cfs.indexManager.flushIndexesBlocking();
 
             finished = writer.finish();
-            cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.CLEANUP);
         }
         finally
         {
@@ -970,11 +969,11 @@ public class CompactionManager implements CompactionManagerMBean
             }
         }
         return SSTableWriter.create(Descriptor.fromFilename(cfs.getTempSSTablePath(compactionFileLocation)),
-                                 (long)expectedBloomFilterSize,
-                                 repairedAt,
-                                 cfs.metadata,
-                                 cfs.partitioner,
-                                 new MetadataCollector(sstables, cfs.metadata.comparator, minLevel));
+                                    (long) expectedBloomFilterSize,
+                                    repairedAt,
+                                    cfs.metadata,
+                                    cfs.partitioner,
+                                    new MetadataCollector(sstables, cfs.metadata.comparator, minLevel));
     }
 
 
@@ -1057,7 +1056,7 @@ public class CompactionManager implements CompactionManagerMBean
             long numPartitions = 0;
             for (SSTableReader sstable : sstables)
             {
-                numPartitions += sstable.estimatedKeysForRanges(Collections.singleton(validator.desc.range));
+                numPartitions += sstable.estimatedKeysForRanges(singleton(validator.desc.range));
             }
             // determine tree depth from number of partitions, but cap at 20 to prevent large tree.
             int depth = numPartitions > 0 ? (int) Math.min(Math.floor(Math.log(numPartitions)), 20) : 0;
@@ -1119,37 +1118,39 @@ public class CompactionManager implements CompactionManagerMBean
      * will store the non-repaired ranges. Once anticompation is completed, the original sstable is marked as compacted
      * and subsequently deleted.
      * @param cfs
-     * @param repairedSSTables
+     * @param repaired a transaction over the repaired sstables to anticompacy
      * @param ranges Repaired ranges to be placed into one of the new sstables. The repaired table will be tracked via
      * the {@link org.apache.cassandra.io.sstable.metadata.StatsMetadata#repairedAt} field.
      */
-    private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
-                                                       Collection<SSTableReader> repairedSSTables, long repairedAt)
+    private void doAntiCompaction(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, LifecycleTransaction repaired, long repairedAt)
     {
-        logger.info("Performing anticompaction on {} sstables", repairedSSTables.size());
+        logger.info("Performing anticompaction on {} sstables", repaired.originals().size());
 
         //Group SSTables
-        Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(repairedSSTables);
+        Collection<Collection<SSTableReader>> groupedSSTables = cfs.getCompactionStrategy().groupSSTablesForAntiCompaction(repaired.originals());
         // iterate over sstables to check if the repaired / unrepaired ranges intersect them.
         int antiCompactedSSTableCount = 0;
         for (Collection<SSTableReader> sstableGroup : groupedSSTables)
         {
-            int antiCompacted = antiCompactGroup(cfs, ranges, sstableGroup, repairedAt);
-            antiCompactedSSTableCount += antiCompacted;
+            try (LifecycleTransaction txn = repaired.split(sstableGroup))
+            {
+                int antiCompacted = antiCompactGroup(cfs, ranges, txn, repairedAt);
+                antiCompactedSSTableCount += antiCompacted;
+            }
         }
 
         String format = "Anticompaction completed successfully, anticompacted from {} to {} sstable(s).";
-        logger.info(format, repairedSSTables.size(), antiCompactedSSTableCount);
+        logger.info(format, repaired.originals().size(), antiCompactedSSTableCount);
     }
 
     private int antiCompactGroup(ColumnFamilyStore cfs, Collection<Range<Token>> ranges,
-                             Collection<SSTableReader> anticompactionGroup, long repairedAt)
+                             LifecycleTransaction anticompactionGroup, long repairedAt)
     {
         long groupMaxDataAge = -1;
 
         // check that compaction hasn't stolen any sstables used in previous repair sessions
         // if we need to skip the anticompaction, it will be carried out by the next repair
-        for (Iterator<SSTableReader> i = anticompactionGroup.iterator(); i.hasNext();)
+        for (Iterator<SSTableReader> i = anticompactionGroup.originals().iterator(); i.hasNext();)
         {
             SSTableReader sstable = i.next();
             if (!new File(sstable.getFilename()).exists())
@@ -1162,26 +1163,25 @@ public class CompactionManager implements CompactionManagerMBean
                 groupMaxDataAge = sstable.maxDataAge;
         }
 
-     
-        if (anticompactionGroup.size() == 0)
+        if (anticompactionGroup.originals().size() == 0)
         {
             logger.info("No valid anticompactions for this group, All sstables were compacted and are no longer available");
             return 0;
         }
 
         logger.info("Anticompacting {}", anticompactionGroup);
-        Set<SSTableReader> sstableAsSet = new HashSet<>(anticompactionGroup);
+        Set<SSTableReader> sstableAsSet = anticompactionGroup.originals();
 
         File destination = cfs.directories.getWriteableLocationAsFile(cfs.getExpectedCompactedFileSize(sstableAsSet, OperationType.ANTICOMPACTION));
         long repairedKeyCount = 0;
         long unrepairedKeyCount = 0;
         AbstractCompactionStrategy strategy = cfs.getCompactionStrategy();
-        try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
-             SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, sstableAsSet, groupMaxDataAge, false);
-             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup);
+        try (SSTableRewriter repairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false);
+             SSTableRewriter unRepairedSSTableWriter = new SSTableRewriter(cfs, anticompactionGroup, groupMaxDataAge, false);
+             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(anticompactionGroup.originals());
              CompactionController controller = new CompactionController(cfs, sstableAsSet, getDefaultGcBefore(cfs)))
         {
-            int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(anticompactionGroup)));
+            int expectedBloomFilterSize = Math.max(cfs.metadata.getMinIndexInterval(), (int)(SSTableReader.getApproximateKeyCount(sstableAsSet)));
 
             repairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, repairedAt, sstableAsSet));
             unRepairedSSTableWriter.switchWriter(CompactionManager.createWriterForAntiCompaction(cfs, destination, expectedBloomFilterSize, ActiveRepairService.UNREPAIRED_SSTABLE, sstableAsSet));
@@ -1212,12 +1212,18 @@ public class CompactionManager implements CompactionManagerMBean
             {
                 metrics.finishCompaction(ci);
             }
-            // we have the same readers being rewritten by both writers, so we ask the first one NOT to close them
-            // so that the second one can do so safely, without leaving us with references < 0 or any other ugliness
+
             List<SSTableReader> anticompactedSSTables = new ArrayList<>();
-            anticompactedSSTables.addAll(repairedSSTableWriter.setRepairedAt(repairedAt).finish());
-            anticompactedSSTables.addAll(unRepairedSSTableWriter.setRepairedAt(ActiveRepairService.UNREPAIRED_SSTABLE).finish());
-            cfs.getDataTracker().markCompactedSSTablesReplaced(sstableAsSet, anticompactedSSTables, OperationType.ANTICOMPACTION);
+            // since both writers are operating over the same Transaction, we cannot use the convenience Transactional.finish() method,
+            // as on the second finish() we would prepareToCommit() on a Transaction that has already been committed, which is forbidden by the API
+            // (since it indicates misuse). We call permitRedundantTransitions so that calls that transition to a state already occupied are permitted.
+            anticompactionGroup.permitRedundantTransitions();
+            repairedSSTableWriter.setRepairedAt(repairedAt).prepareToCommit();
+            unRepairedSSTableWriter.prepareToCommit();
+            anticompactedSSTables.addAll(repairedSSTableWriter.finished());
+            anticompactedSSTables.addAll(unRepairedSSTableWriter.finished());
+            repairedSSTableWriter.commit();
+            unRepairedSSTableWriter.commit();
 
             logger.debug("Repaired {} keys out of {} for {}/{} in {}", repairedKeyCount,
                                                                        repairedKeyCount + unrepairedKeyCount,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 34f57c1..e593ec0 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.io.File;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -44,6 +42,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorStatsCollector;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.UUIDGen;
 
@@ -55,9 +54,9 @@ public class CompactionTask extends AbstractCompactionTask
     protected static long totalBytesCompacted = 0;
     private CompactionExecutorStatsCollector collector;
 
-    public CompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int gcBefore, boolean offline)
+    public CompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
     {
-        super(cfs, Sets.newHashSet(sstables));
+        super(cfs, txn);
         this.gcBefore = gcBefore;
         this.offline = offline;
     }
@@ -71,23 +70,20 @@ public class CompactionTask extends AbstractCompactionTask
     {
         this.collector = collector;
         run();
-        return sstables.size();
+        return transaction.originals().size();
     }
 
     public boolean reduceScopeForLimitedSpace()
     {
-        if (partialCompactionsAcceptable() && sstables.size() > 1)
+        if (partialCompactionsAcceptable() && transaction.originals().size() > 1)
         {
             // Try again w/o the largest one.
-            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(sstables, ", "));
+            logger.warn("insufficient space to compact all requested files {}", StringUtils.join(transaction.originals(), ", "));
             // Note that we have removed files that are still marked as compacting.
             // This suboptimal but ok since the caller will unmark all the sstables at the end.
-            SSTableReader removedSSTable = cfs.getMaxSizeFile(sstables);
-            if (sstables.remove(removedSSTable))
-            {
-                cfs.getDataTracker().unmarkCompacting(Arrays.asList(removedSSTable));
-                return true;
-            }
+            SSTableReader removedSSTable = cfs.getMaxSizeFile(transaction.originals());
+            transaction.cancel(removedSSTable);
+            return true;
         }
         return false;
     }
@@ -101,9 +97,9 @@ public class CompactionTask extends AbstractCompactionTask
     {
         // The collection of sstables passed may be empty (but not null); even if
         // it is not empty, it may compact down to nothing if all rows are deleted.
-        assert sstables != null;
+        assert transaction != null;
 
-        if (sstables.size() == 0)
+        if (transaction.originals().isEmpty())
             return;
 
         // Note that the current compaction strategy, is not necessarily the one this task was created under.
@@ -115,12 +111,12 @@ public class CompactionTask extends AbstractCompactionTask
 
         // note that we need to do a rough estimate early if we can fit the compaction on disk - this is pessimistic, but
         // since we might remove sstables from the compaction in checkAvailableDiskSpace it needs to be done here
-        long expectedWriteSize = cfs.getExpectedCompactedFileSize(sstables, compactionType);
+        long expectedWriteSize = cfs.getExpectedCompactedFileSize(transaction.originals(), compactionType);
         long earlySSTableEstimate = Math.max(1, expectedWriteSize / strategy.getMaxSSTableBytes());
         checkAvailableDiskSpace(earlySSTableEstimate, expectedWriteSize);
 
         // sanity check: all sstables must belong to the same cfs
-        assert !Iterables.any(sstables, new Predicate<SSTableReader>()
+        assert !Iterables.any(transaction.originals(), new Predicate<SSTableReader>()
         {
             @Override
             public boolean apply(SSTableReader sstable)
@@ -129,13 +125,13 @@ public class CompactionTask extends AbstractCompactionTask
             }
         });
 
-        UUID taskId = SystemKeyspace.startCompaction(cfs, sstables);
+        UUID taskId = SystemKeyspace.startCompaction(cfs, transaction.originals());
 
         // new sstables from flush can be added during a compaction, but only the compaction can remove them,
         // so in our single-threaded compaction world this is a valid way of determining if we're compacting
         // all the sstables (that existed when we started)
         StringBuilder ssTableLoggerMsg = new StringBuilder("[");
-        for (SSTableReader sstr : sstables)
+        for (SSTableReader sstr : transaction.originals())
         {
             ssTableLoggerMsg.append(String.format("%s:level=%d, ", sstr.getFilename(), sstr.getSSTableLevel()));
         }
@@ -148,11 +144,11 @@ public class CompactionTask extends AbstractCompactionTask
         long totalKeysWritten = 0;
 
         long estimatedKeys = 0;
-        try (CompactionController controller = getCompactionController(sstables))
+        try (CompactionController controller = getCompactionController(transaction.originals()))
         {
-            Set<SSTableReader> actuallyCompact = Sets.difference(sstables, controller.getFullyExpiredSSTables());
+            Set<SSTableReader> actuallyCompact = Sets.difference(transaction.originals(), controller.getFullyExpiredSSTables());
 
-            SSTableFormat.Type sstableFormat = getFormatType(sstables);
+            SSTableFormat.Type sstableFormat = getFormatType(transaction.originals());
 
             List<SSTableReader> newSStables;
             AbstractCompactionIterable ci;
@@ -171,7 +167,7 @@ public class CompactionTask extends AbstractCompactionTask
                 if (!controller.cfs.getCompactionStrategy().isActive)
                     throw new CompactionInterruptedException(ci.getCompactionInfo());
 
-                try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, sstables, actuallyCompact))
+                try (CompactionAwareWriter writer = getCompactionAwareWriter(cfs, transaction, actuallyCompact))
                 {
                     estimatedKeys = writer.estimatedKeys();
                     while (iter.hasNext())
@@ -205,13 +201,9 @@ public class CompactionTask extends AbstractCompactionTask
                 }
             }
 
-            Collection<SSTableReader> oldSStables = this.sstables;
-            if (!offline)
-                cfs.getDataTracker().markCompactedSSTablesReplaced(oldSStables, newSStables, compactionType);
-
             // log a bunch of statistics about the result and save to system table compaction_history
             long dTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start);
-            long startsize = SSTableReader.getTotalBytes(oldSStables);
+            long startsize = SSTableReader.getTotalBytes(transaction.originals());
             long endsize = SSTableReader.getTotalBytes(newSStables);
             double ratio = (double) endsize / (double) startsize;
 
@@ -223,7 +215,7 @@ public class CompactionTask extends AbstractCompactionTask
             long totalSourceRows = 0;
             String mergeSummary = updateCompactionHistory(cfs.keyspace.getName(), cfs.getColumnFamilyName(), ci, startsize, endsize);
             logger.info(String.format("Compacted (%s) %d sstables to [%s] to level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total partitions merged to %,d.  Partition merge counts were {%s}",
-                                      taskIdLoggerMsg, oldSStables.size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
+                                      taskIdLoggerMsg, transaction.originals().size(), newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
             logger.debug(String.format("CF Total Bytes Compacted: %,d", CompactionTask.addToTotalBytesCompacted(endsize)));
             logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - estimatedKeys)/totalKeysWritten));
 
@@ -236,9 +228,9 @@ public class CompactionTask extends AbstractCompactionTask
     }
 
     @Override
-    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, Set<SSTableReader> nonExpiredSSTables)
     {
-        return new DefaultCompactionWriter(cfs, allSSTables, nonExpiredSSTables, offline, compactionType);
+        return new DefaultCompactionWriter(cfs, transaction, nonExpiredSSTables, offline, compactionType);
 
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
index 6385671..18d5f7b 100644
--- a/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/DateTieredCompactionStrategy.java
@@ -28,6 +28,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.statements.CFPropDefs;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.Pair;
 
@@ -67,8 +68,9 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
             if (latestBucket.isEmpty())
                 return null;
 
-            if (cfs.getDataTracker().markCompacting(latestBucket))
-                return new CompactionTask(cfs, latestBucket, gcBefore, false);
+            LifecycleTransaction modifier = cfs.getTracker().tryModify(latestBucket, OperationType.COMPACTION);
+            if (modifier != null)
+                return new CompactionTask(cfs, modifier, gcBefore, false);
         }
     }
 
@@ -366,11 +368,11 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
     @Override
     public synchronized Collection<AbstractCompactionTask> getMaximalTask(int gcBefore, boolean splitOutput)
     {
-        Iterable<SSTableReader> sstables = cfs.markAllCompacting();
-        if (sstables == null)
+        LifecycleTransaction modifier = cfs.markAllCompacting(OperationType.COMPACTION);
+        if (modifier == null)
             return null;
 
-        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, sstables, gcBefore, false));
+        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, modifier, gcBefore, false));
     }
 
     @Override
@@ -378,13 +380,14 @@ public class DateTieredCompactionStrategy extends AbstractCompactionStrategy
     {
         assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
 
-        if (!cfs.getDataTracker().markCompacting(sstables))
+        LifecycleTransaction modifier = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+        if (modifier == null)
         {
             logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 
-        return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+        return new CompactionTask(cfs, modifier, gcBefore, false).setUserDefined(true);
     }
 
     public int getEstimatedRemainingTasks()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
index 6b82ad3..c434d31 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionStrategy.java
@@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -115,9 +116,10 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
                 op = OperationType.COMPACTION;
             }
 
-            if (cfs.getDataTracker().markCompacting(candidate.sstables))
+            LifecycleTransaction txn = cfs.getTracker().tryModify(candidate.sstables, OperationType.COMPACTION);
+            if (txn != null)
             {
-                LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, candidate.sstables, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
+                LeveledCompactionTask newTask = new LeveledCompactionTask(cfs, txn, candidate.level, gcBefore, candidate.maxSSTableBytes, false);
                 newTask.setCompactionType(op);
                 return newTask;
             }
@@ -131,9 +133,10 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
         Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
         if (Iterables.isEmpty(sstables))
             return null;
-        if (!cfs.getDataTracker().markCompacting(filteredSSTables))
+        LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+        if (txn == null)
             return null;
-        return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, filteredSSTables, 0, gcBefore, getMaxSSTableBytes(), true));
+        return Arrays.<AbstractCompactionTask>asList(new LeveledCompactionTask(cfs, txn, 0, gcBefore, getMaxSSTableBytes(), true));
 
     }
 
@@ -144,19 +147,19 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
     }
 
     @Override
-    public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, int gcBefore, long maxSSTableBytes)
+    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, int gcBefore, long maxSSTableBytes)
     {
-        assert sstables.size() > 0;
+        assert txn.originals().size() > 0;
         int level = -1;
         // if all sstables are in the same level, we can set that level:
-        for (SSTableReader sstable : sstables)
+        for (SSTableReader sstable : txn.originals())
         {
             if (level == -1)
                 level = sstable.getSSTableLevel();
             if (level != sstable.getSSTableLevel())
                 level = 0;
         }
-        return new LeveledCompactionTask(cfs, sstables, level, gcBefore, maxSSTableBytes, false);
+        return new LeveledCompactionTask(cfs, txn, level, gcBefore, maxSSTableBytes, false);
     }
 
     /**
@@ -226,7 +229,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
         {
             for (Integer level : byLevel.keySet())
             {
-                // level can be -1 when sstables are added to DataTracker but not to LeveledManifest
+                // level can be -1 when sstables are added to Tracker but not to LeveledManifest
                 // since we don't know which level those sstable belong yet, we simply do the same as L0 sstables.
                 if (level <= 0)
                 {
@@ -402,7 +405,7 @@ public class LeveledCompactionStrategy extends AbstractCompactionStrategy
             if (sstables.isEmpty())
                 continue;
 
-            Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+            Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
             for (SSTableReader sstable : sstables)
             {
                 if (sstable.getEstimatedDroppableTombstoneRatio(gcBefore) <= tombstoneThreshold)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
index ce9dfaf..1c3b686 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledCompactionTask.java
@@ -24,6 +24,7 @@ import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
 import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
 import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
 public class LeveledCompactionTask extends CompactionTask
 {
@@ -31,20 +32,20 @@ public class LeveledCompactionTask extends CompactionTask
     private final long maxSSTableBytes;
     private final boolean majorCompaction;
 
-    public LeveledCompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int level, final int gcBefore, long maxSSTableBytes, boolean majorCompaction)
+    public LeveledCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int level, int gcBefore, long maxSSTableBytes, boolean majorCompaction)
     {
-        super(cfs, sstables, gcBefore, false);
+        super(cfs, txn, gcBefore, false);
         this.level = level;
         this.maxSSTableBytes = maxSSTableBytes;
         this.majorCompaction = majorCompaction;
     }
 
     @Override
-    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+    public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
     {
         if (majorCompaction)
-            return new MajorLeveledCompactionWriter(cfs, sstables, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
-        return new MaxSSTableSizeWriter(cfs, sstables, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
+            return new MajorLeveledCompactionWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, false, compactionType);
+        return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, maxSSTableBytes, getLevel(), false, compactionType);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index daff131..0d0928f 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -23,7 +23,6 @@ import java.util.*;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
@@ -316,7 +315,7 @@ public class LeveledManifest
                 continue; // mostly this just avoids polluting the debug log with zero scores
             // we want to calculate score excluding compacting ones
             Set<SSTableReader> sstablesInLevel = Sets.newHashSet(sstables);
-            Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getDataTracker().getCompacting());
+            Set<SSTableReader> remaining = Sets.difference(sstablesInLevel, cfs.getTracker().getCompacting());
             double score = (double) SSTableReader.getTotalBytes(remaining) / (double)maxBytesForLevel(i, maxSSTableSizeInBytes);
             logger.debug("Compaction score for level {} is {}", i, score);
 
@@ -361,7 +360,7 @@ public class LeveledManifest
 
     private List<SSTableReader> getSSTablesForSTCS(Collection<SSTableReader> sstables)
     {
-        Iterable<SSTableReader> candidates = cfs.getDataTracker().getUncompactingSSTables(sstables);
+        Iterable<SSTableReader> candidates = cfs.getTracker().getUncompacting(sstables);
         List<Pair<SSTableReader,Long>> pairs = SizeTieredCompactionStrategy.createSSTableAndLengthPairs(AbstractCompactionStrategy.filterSuspectSSTables(candidates));
         List<List<SSTableReader>> buckets = SizeTieredCompactionStrategy.getBuckets(pairs,
                                                                                     options.bucketHigh,
@@ -415,7 +414,7 @@ public class LeveledManifest
                     }
                     if (min == null || max == null || min.equals(max)) // single partition sstables - we cannot include a high level sstable.
                         return candidates;
-                    Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+                    Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
                     Range<RowPosition> boundaries = new Range<>(min, max);
                     for (SSTableReader sstable : getLevel(i))
                     {
@@ -542,7 +541,7 @@ public class LeveledManifest
         assert !getLevel(level).isEmpty();
         logger.debug("Choosing candidates for L{}", level);
 
-        final Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
+        final Set<SSTableReader> compacting = cfs.getTracker().getCompacting();
 
         if (level == 0)
         {
@@ -650,7 +649,7 @@ public class LeveledManifest
     {
         Set<SSTableReader> sstables = new HashSet<>();
         Set<SSTableReader> levelSSTables = new HashSet<>(getLevel(level));
-        for (SSTableReader sstable : cfs.getDataTracker().getCompacting())
+        for (SSTableReader sstable : cfs.getTracker().getCompacting())
         {
             if (levelSSTables.contains(sstable))
                 sstables.add(sstable);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
index 8d7b0e9..e9a4f05 100644
--- a/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
+++ b/src/java/org/apache/cassandra/db/compaction/SSTableSplitter.java
@@ -23,6 +23,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
 import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
 public class SSTableSplitter {
 
@@ -30,9 +31,9 @@ public class SSTableSplitter {
 
     private CompactionInfo.Holder info;
 
-    public SSTableSplitter(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+    public SSTableSplitter(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
     {
-        this.task = new SplittingCompactionTask(cfs, sstable, sstableSizeInMB);
+        this.task = new SplittingCompactionTask(cfs, transaction, sstableSizeInMB);
     }
 
     public void split()
@@ -57,9 +58,9 @@ public class SSTableSplitter {
     {
         private final int sstableSizeInMB;
 
-        public SplittingCompactionTask(ColumnFamilyStore cfs, SSTableReader sstable, int sstableSizeInMB)
+        public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction, int sstableSizeInMB)
         {
-            super(cfs, Collections.singletonList(sstable), CompactionManager.NO_GC, true);
+            super(cfs, transaction, CompactionManager.NO_GC, true);
             this.sstableSizeInMB = sstableSizeInMB;
 
             if (sstableSizeInMB <= 0)
@@ -73,9 +74,9 @@ public class SSTableSplitter {
         }
 
         @Override
-        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
         {
-            return new MaxSSTableSizeWriter(cfs, sstables, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
+            return new MaxSSTableSizeWriter(cfs, txn, nonExpiredSSTables, sstableSizeInMB * 1024L * 1024L, 0, true, compactionType);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 1e014ed..b7c149c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -23,9 +23,9 @@ import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -41,6 +41,7 @@ public class Scrubber implements Closeable
 {
     private final ColumnFamilyStore cfs;
     private final SSTableReader sstable;
+    private final LifecycleTransaction transaction;
     private final File destination;
     private final boolean skipCorrupted;
 
@@ -80,15 +81,16 @@ public class Scrubber implements Closeable
     };
     private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
     {
         this.cfs = cfs;
-        this.sstable = sstable;
+        this.transaction = transaction;
+        this.sstable = transaction.onlyOne();
         this.outputHandler = outputHandler;
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
@@ -127,9 +129,7 @@ public class Scrubber implements Closeable
     public void scrub()
     {
         outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
-        Set<SSTableReader> oldSSTable = Sets.newHashSet(sstable);
-
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, oldSSTable, sstable.maxDataAge, isOffline);)
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, isOffline);)
         {
             nextIndexKey = ByteBufferUtil.readWithShortLength(indexFile);
             {
@@ -278,8 +278,7 @@ public class Scrubber implements Closeable
                         inOrderWriter.append(row.key, row.cf);
                     newInOrderSstable = inOrderWriter.finish(-1, sstable.maxDataAge, true);
                 }
-                if (!isOffline)
-                    cfs.getDataTracker().addSSTables(Collections.singleton(newInOrderSstable));
+                transaction.update(newInOrderSstable, false);
                 outputHandler.warn(String.format("%d out of order rows found while scrubbing %s; Those have been written (in order) to a new sstable (%s)", outOfOrderRows.size(), sstable, newInOrderSstable));
             }
 
@@ -287,8 +286,6 @@ public class Scrubber implements Closeable
             List<SSTableReader> finished = writer.setRepairedAt(badRows > 0 ? ActiveRepairService.UNREPAIRED_SSTABLE : sstable.getSSTableMetadata().repairedAt).finish();
             if (!finished.isEmpty())
                 newSstable = finished.get(0);
-            if (!isOffline)
-                cfs.getDataTracker().markCompactedSSTablesReplaced(oldSSTable, finished, OperationType.SCRUB);
         }
         catch (IOException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
index 722536c..94c3daf 100644
--- a/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategy.java
@@ -21,22 +21,19 @@ import java.util.*;
 import java.util.Map.Entry;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
 import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import com.google.common.primitives.Longs;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.statements.CFPropDefs;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.ColumnNameHelper;
 import org.apache.cassandra.utils.Pair;
 
 public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
@@ -190,8 +187,9 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
             if (hottestBucket.isEmpty())
                 return null;
 
-            if (cfs.getDataTracker().markCompacting(hottestBucket))
-                return new CompactionTask(cfs, hottestBucket, gcBefore, false);
+            LifecycleTransaction transaction = cfs.getTracker().tryModify(hottestBucket, OperationType.COMPACTION);
+            if (transaction != null)
+                return new CompactionTask(cfs, transaction, gcBefore, false);
         }
     }
 
@@ -200,24 +198,26 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
         Iterable<SSTableReader> filteredSSTables = filterSuspectSSTables(sstables);
         if (Iterables.isEmpty(filteredSSTables))
             return null;
-        if (!cfs.getDataTracker().markCompacting(ImmutableList.copyOf(filteredSSTables)))
+        LifecycleTransaction txn = cfs.getTracker().tryModify(filteredSSTables, OperationType.COMPACTION);
+        if (txn == null)
             return null;
         if (splitOutput)
-            return Arrays.<AbstractCompactionTask>asList(new SplittingCompactionTask(cfs, filteredSSTables, gcBefore, false));
-        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, filteredSSTables, gcBefore, false));
+            return Arrays.<AbstractCompactionTask>asList(new SplittingCompactionTask(cfs, txn, gcBefore, false));
+        return Arrays.<AbstractCompactionTask>asList(new CompactionTask(cfs, txn, gcBefore, false));
     }
 
     public AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore)
     {
         assert !sstables.isEmpty(); // checked for by CM.submitUserDefined
 
-        if (!cfs.getDataTracker().markCompacting(sstables))
+        LifecycleTransaction transaction = cfs.getTracker().tryModify(sstables, OperationType.COMPACTION);
+        if (transaction == null)
         {
             logger.debug("Unable to mark {} for compaction; probably a background compaction got to it first.  You can disable background compactions temporarily if this is a problem", sstables);
             return null;
         }
 
-        return new CompactionTask(cfs, sstables, gcBefore, false).setUserDefined(true);
+        return new CompactionTask(cfs, transaction, gcBefore, false).setUserDefined(true);
     }
 
     public int getEstimatedRemainingTasks()
@@ -338,15 +338,15 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy
 
     private static class SplittingCompactionTask extends CompactionTask
     {
-        public SplittingCompactionTask(ColumnFamilyStore cfs, Iterable<SSTableReader> sstables, int gcBefore, boolean offline)
+        public SplittingCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction txn, int gcBefore, boolean offline)
         {
-            super(cfs, sstables, gcBefore, offline);
+            super(cfs, txn, gcBefore, offline);
         }
 
         @Override
-        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables)
+        public CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables)
         {
-            return new SplittingSizeTieredCompactionWriter(cfs, allSSTables, nonExpiredSSTables, compactionType);
+            return new SplittingSizeTieredCompactionWriter(cfs, txn, nonExpiredSSTables, compactionType);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/Upgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Upgrader.java b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
index 5bb1530..6556a71 100644
--- a/src/java/org/apache/cassandra/db/compaction/Upgrader.java
+++ b/src/java/org/apache/cassandra/db/compaction/Upgrader.java
@@ -20,12 +20,10 @@ package org.apache.cassandra.db.compaction;
 import java.io.File;
 import java.util.*;
 
-import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
-
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -37,6 +35,7 @@ public class Upgrader
 {
     private final ColumnFamilyStore cfs;
     private final SSTableReader sstable;
+    private final LifecycleTransaction transaction;
     private final File directory;
 
     private final OperationType compactionType = OperationType.UPGRADE_SSTABLES;
@@ -46,10 +45,11 @@ public class Upgrader
 
     private final OutputHandler outputHandler;
 
-    public Upgrader(ColumnFamilyStore cfs, SSTableReader sstable, OutputHandler outputHandler)
+    public Upgrader(ColumnFamilyStore cfs, LifecycleTransaction txn, OutputHandler outputHandler)
     {
         this.cfs = cfs;
-        this.sstable = sstable;
+        this.transaction = txn;
+        this.sstable = txn.onlyOne();
         this.outputHandler = outputHandler;
 
         this.directory = new File(sstable.getFilename()).getParentFile();
@@ -81,10 +81,9 @@ public class Upgrader
     public void upgrade()
     {
         outputHandler.output("Upgrading " + sstable);
-        Set<SSTableReader> toUpgrade = Sets.newHashSet(sstable);
 
-        try (SSTableRewriter writer = new SSTableRewriter(cfs, toUpgrade, CompactionTask.getMaxDataAge(toUpgrade), true);
-             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(toUpgrade))
+        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, CompactionTask.getMaxDataAge(transaction.originals()), true);
+             AbstractCompactionStrategy.ScannerList scanners = strategy.getScanners(transaction.originals()))
         {
             Iterator<AbstractCompactedRow> iter = new CompactionIterable(compactionType, scanners.scanners, controller, DatabaseDescriptor.getSSTableFormat(), UUIDGen.getTimeUUID()).iterator();
             writer.switchWriter(createCompactionWriter(sstable.getSSTableMetadata().repairedAt));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
index 5345d8d..c511bcd 100644
--- a/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/WrappingCompactionStrategy.java
@@ -50,7 +50,7 @@ public final class WrappingCompactionStrategy extends AbstractCompactionStrategy
     {
         super(cfs, cfs.metadata.compactionStrategyOptions);
         reloadCompactionStrategy(cfs.metadata);
-        cfs.getDataTracker().subscribe(this);
+        cfs.getTracker().subscribe(this);
         logger.debug("{} subscribed to the data tracker.", this);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
index fe43186..20c96d6 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/CompactionAwareWriter.java
@@ -25,6 +25,7 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
 import org.apache.cassandra.db.compaction.CompactionTask;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.SSTableRewriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.concurrent.Transactional;
@@ -43,14 +44,14 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
     protected final long minRepairedAt;
     protected final SSTableRewriter sstableWriter;
 
-    public CompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline)
+    public CompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline)
     {
         this.cfs = cfs;
         this.nonExpiredSSTables = nonExpiredSSTables;
         this.estimatedTotalKeys = SSTableReader.getApproximateKeyCount(nonExpiredSSTables);
         this.maxAge = CompactionTask.getMaxDataAge(nonExpiredSSTables);
         this.minRepairedAt = CompactionTask.getMinRepairedAt(nonExpiredSSTables);
-        this.sstableWriter = new SSTableRewriter(cfs, allSSTables, maxAge, offline);
+        this.sstableWriter = new SSTableRewriter(cfs, txn, maxAge, offline);
     }
 
     /**
@@ -67,12 +68,6 @@ public abstract class CompactionAwareWriter extends Transactional.AbstractTransa
     }
 
     @Override
-    protected Throwable doCleanup(Throwable accumulate)
-    {
-        return accumulate;
-    }
-
-    @Override
     protected Throwable doCommit(Throwable accumulate)
     {
         return sstableWriter.commit(accumulate);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
index 3589b54..0b31061 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/DefaultCompactionWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction.writers;
 
 
 import java.io.File;
-import java.util.List;
 import java.util.Set;
 
 import org.slf4j.Logger;
@@ -28,14 +27,12 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 
-import static org.apache.cassandra.utils.Throwables.maybeFail;
-
 
 /**
  * The default compaction writer - creates one output file in L0
@@ -44,9 +41,9 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
 {
     protected static final Logger logger = LoggerFactory.getLogger(DefaultCompactionWriter.class);
 
-    public DefaultCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
+    public DefaultCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, boolean offline, OperationType compactionType)
     {
-        super(cfs, allSSTables, nonExpiredSSTables, offline);
+        super(cfs, txn, nonExpiredSSTables, offline);
         logger.debug("Expected bloom filter size : {}", estimatedTotalKeys);
         long expectedWriteSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
         File sstableDirectory = cfs.directories.getLocationForDisk(getWriteDirectory(expectedWriteSize));
@@ -55,7 +52,7 @@ public class DefaultCompactionWriter extends CompactionAwareWriter
                                                     minRepairedAt,
                                                     cfs.metadata,
                                                     cfs.partitioner,
-                                                    new MetadataCollector(allSSTables, cfs.metadata.comparator, 0));
+                                                    new MetadataCollector(txn.originals(), cfs.metadata.comparator, 0));
         sstableWriter.switchWriter(writer);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
index d48140e..014b4af 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MajorLeveledCompactionWriter.java
@@ -18,8 +18,6 @@
 package org.apache.cassandra.db.compaction.writers;
 
 import java.io.File;
-import java.util.Collections;
-import java.util.List;
 import java.util.Set;
 
 import org.slf4j.Logger;
@@ -28,11 +26,10 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
 import org.apache.cassandra.db.compaction.LeveledManifest;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -50,11 +47,11 @@ public class MajorLeveledCompactionWriter extends CompactionAwareWriter
     private int sstablesWritten = 0;
     private final boolean skipAncestors;
 
-    public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
+    public MajorLeveledCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, boolean offline, OperationType compactionType)
     {
-        super(cfs, allSSTables, nonExpiredSSTables, offline);
+        super(cfs, txn, nonExpiredSSTables, offline);
         this.maxSSTableSize = maxSSTableSize;
-        this.allSSTables = allSSTables;
+        this.allSSTables = txn.originals();
         expectedWriteSize = Math.min(maxSSTableSize, cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType));
         long estimatedSSTables = Math.max(1, SSTableReader.getTotalBytes(nonExpiredSSTables) / maxSSTableSize);
         long keysPerSSTable = estimatedTotalKeys / estimatedSSTables;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
index ab24bf8..8903ff7 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/MaxSSTableSizeWriter.java
@@ -18,16 +18,14 @@
 package org.apache.cassandra.db.compaction.writers;
 
 import java.io.File;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -41,10 +39,10 @@ public class MaxSSTableSizeWriter extends CompactionAwareWriter
     private final long estimatedSSTables;
     private final Set<SSTableReader> allSSTables;
 
-    public MaxSSTableSizeWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
+    public MaxSSTableSizeWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, long maxSSTableSize, int level, boolean offline, OperationType compactionType)
     {
-        super(cfs, allSSTables, nonExpiredSSTables, offline);
-        this.allSSTables = allSSTables;
+        super(cfs, txn, nonExpiredSSTables, offline);
+        this.allSSTables = txn.originals();
         this.level = level;
         this.maxSSTableSize = maxSSTableSize;
         long totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index 2a452c7..81ea6b1 100644
--- a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction.writers;
 
 import java.io.File;
 import java.util.Arrays;
-import java.util.List;
 import java.util.Set;
 
 import org.slf4j.Logger;
@@ -28,10 +27,9 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
-import org.apache.cassandra.db.compaction.CompactionTask;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableRewriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -53,15 +51,15 @@ public class SplittingSizeTieredCompactionWriter extends CompactionAwareWriter
     private long currentBytesToWrite;
     private int currentRatioIndex = 0;
 
-    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
+    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType)
     {
-        this(cfs, allSSTables, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
+        this(cfs, txn, nonExpiredSSTables, compactionType, DEFAULT_SMALLEST_SSTABLE_BYTES);
     }
 
-    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
+    public SplittingSizeTieredCompactionWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables, OperationType compactionType, long smallestSSTable)
     {
-        super(cfs, allSSTables, nonExpiredSSTables, false);
-        this.allSSTables = allSSTables;
+        super(cfs, txn, nonExpiredSSTables, false);
+        this.allSSTables = txn.originals();
         totalSize = cfs.getExpectedCompactedFileSize(nonExpiredSSTables, compactionType);
         double[] potentialRatios = new double[20];
         double currentRatio = 1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index f8b3aba..ba48350 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -57,7 +57,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
                                                              indexedCfMetadata.cfName,
                                                              new LocalPartitioner(getIndexKeyComparator()),
                                                              indexedCfMetadata,
-                                                             baseCfs.getDataTracker().loadsstables);
+                                                             baseCfs.getTracker().loadsstables);
     }
 
     protected AbstractType<?> getIndexKeyComparator()
@@ -143,7 +143,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
     {
         Future<?> wait;
         // we synchronise on the baseCfs to make sure we are ordered correctly with other flushes to the base CFS
-        synchronized (baseCfs.getDataTracker())
+        synchronized (baseCfs.getTracker())
         {
             wait = indexCfs.forceFlush();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index dda532d..4c1bf45 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -346,7 +346,7 @@ public class SecondaryIndexManager
     {
         // despatch flushes for all CFS backed indexes
         List<Future<?>> wait = new ArrayList<>();
-        synchronized (baseCfs.getDataTracker())
+        synchronized (baseCfs.getTracker())
         {
             for (SecondaryIndex index : allIndexes)
                 if (index.getIndexCfs() != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Helpers.java b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
new file mode 100644
index 0000000..05f7531
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Helpers.java
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.Iterables.any;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static com.google.common.collect.Iterables.getFirst;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+class Helpers
+{
+    /**
+     * update the contents of a set with the provided sets, ensuring that the items to remove are
+     * really present, and that the items to add are not (unless we're also removing them)
+     * @return a new set with the contents of the provided one modified
+     */
+    static <T> Set<T> replace(Set<T> original, Set<T> remove, Iterable<T> add)
+    {
+        return ImmutableSet.copyOf(replace(identityMap(original), remove, add).keySet());
+    }
+
+    /**
+     * update the contents of an "identity map" with the provided sets, ensuring that the items to remove are
+     * really present, and that the items to add are not (unless we're also removing them)
+     * @return a new identity map with the contents of the provided one modified
+     */
+    static <T> Map<T, T> replace(Map<T, T> original, Set<T> remove, Iterable<T> add)
+    {
+        // ensure the ones being removed are the exact same ones present
+        for (T reader : remove)
+            assert original.get(reader) == reader;
+
+        // ensure we don't already contain any we're adding, that we aren't also removing
+        assert !any(add, and(not(in(remove)), in(original.keySet()))) : String.format("original:%s remove:%s add:%s", original.keySet(), remove, add);
+
+        Map<T, T> result =
+            identityMap(concat(add, filter(original.keySet(), not(in(remove)))));
+
+        assert result.size() == original.size() - remove.size() + Iterables.size(add) :
+        String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
+                      original.size() - remove.size() + Iterables.size(add), result.size(), remove, add, original.keySet());
+        return result;
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static Throwable setupDeleteNotification(Iterable<SSTableReader> readers, Tracker tracker, Throwable accumulate)
+    {
+        try
+        {
+            for (SSTableReader reader : readers)
+                reader.setupDeleteNotification(tracker);
+        }
+        catch (Throwable t)
+        {
+            // shouldn't be possible, but in case the contract changes in future and we miss it...
+            accumulate = merge(accumulate, t);
+        }
+        return accumulate;
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static Throwable setReplaced(Iterable<SSTableReader> readers, Throwable accumulate)
+    {
+        for (SSTableReader reader : readers)
+        {
+            try
+            {
+                reader.setReplaced();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    /**
+     * assert that none of these readers have been replaced
+     */
+    static void checkNotReplaced(Iterable<SSTableReader> readers)
+    {
+        for (SSTableReader reader : readers)
+            assert !reader.isReplaced();
+    }
+
+    /**
+     * A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
+     * @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
+     */
+    static Throwable markObsolete(Iterable<SSTableReader> readers, Throwable accumulate)
+    {
+        for (SSTableReader reader : readers)
+        {
+            try
+            {
+                boolean firstToCompact = reader.markObsolete();
+                assert firstToCompact : reader + " was already marked compacted";
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    /**
+     * @return the identity function, as a Map, with domain of the provided values
+     */
+    static <T> Map<T, T> identityMap(Iterable<T> values)
+    {
+        ImmutableMap.Builder<T, T> builder = ImmutableMap.<T, T>builder();
+        for (T t : values)
+            builder.put(t, t);
+        return builder.build();
+    }
+
+    /**
+     * @return an Iterable of the union if the sets, with duplicates being represented by their first encountered instance
+     * (as defined by the order of set provision)
+     */
+    static <T> Iterable<T> concatUniq(Set<T>... sets)
+    {
+        List<Predicate<T>> notIn = new ArrayList<>(sets.length);
+        for (Set<T> set : sets)
+            notIn.add(not(in(set)));
+        List<Iterable<T>> results = new ArrayList<>(sets.length);
+        for (int i = 0 ; i < sets.length ; i++)
+            results.add(filter(sets[i], and(notIn.subList(0, i))));
+        return concat(results);
+    }
+
+    /**
+     * @return a Predicate yielding true for an item present in NONE of the provided sets
+     */
+    static <T> Predicate<T> notIn(Set<T>... sets)
+    {
+        return not(orIn(sets));
+    }
+
+    /**
+     * @return a Predicate yielding true for an item present in ANY of the provided sets
+     */
+    static <T> Predicate<T> orIn(Collection<T>... sets)
+    {
+        Predicate<T>[] orIn = new Predicate[sets.length];
+        for (int i = 0 ; i < orIn.length ; i++)
+            orIn[i] = in(sets[i]);
+        return or(orIn);
+    }
+
+    /**
+     * filter out (i.e. remove) matching elements
+     * @return filter, filtered to only those elements that *are not* present in *any* of the provided sets (are present in none)
+     */
+    static <T> Iterable<T> filterOut(Iterable<T> filter, Set<T>... inNone)
+    {
+        return filter(filter, notIn(inNone));
+    }
+
+    /**
+     * filter in (i.e. retain)
+     *
+     * @return filter, filtered to only those elements that *are* present in *any* of the provided sets
+     */
+    static <T> Iterable<T> filterIn(Iterable<T> filter, Set<T>... inAny)
+    {
+        return filter(filter, orIn(inAny));
+    }
+
+    static Set<SSTableReader> emptySet()
+    {
+        return Collections.emptySet();
+    }
+
+    static <T> T select(T t, Collection<T> col)
+    {
+        if (col instanceof Set && !col.contains(t))
+            return null;
+        return getFirst(filter(col, equalTo(t)), null);
+    }
+
+    static <T> T selectFirst(T t, Collection<T> ... sets)
+    {
+        for (Collection<T> set : sets)
+        {
+            T select = select(t, set);
+            if (select != null)
+                return select;
+        }
+        return null;
+    }
+
+    static <T> Predicate<T> idIn(Set<T> set)
+    {
+        return idIn(identityMap(set));
+    }
+
+    static <T> Predicate<T> idIn(final Map<T, T> identityMap)
+    {
+        return new Predicate<T>()
+        {
+            public boolean apply(T t)
+            {
+                return identityMap.get(t) == t;
+            }
+        };
+    }
+
+}


[4/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
new file mode 100644
index 0000000..acc9747
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/LifecycleTransaction.java
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.format.SSTableReader.UniqueIdentifier;
+import org.apache.cassandra.utils.concurrent.Transactional;
+
+import static com.google.common.base.Functions.compose;
+import static com.google.common.base.Predicates.*;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.*;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+public class LifecycleTransaction extends Transactional.AbstractTransactional
+{
+    private static final Logger logger = LoggerFactory.getLogger(LifecycleTransaction.class);
+
+    /**
+     * a class that represents accumulated modifications to the Tracker.
+     * has two instances, one containing modifications that are "staged" (i.e. invisible)
+     * and one containing those "logged" that have been made visible through a call to checkpoint()
+     */
+    private static class State
+    {
+        // readers that are either brand new, update a previous new reader, or update one of the original readers
+        final Set<SSTableReader> update = new HashSet<>();
+        // disjoint from update, represents a subset of originals that is no longer needed
+        final Set<SSTableReader> obsolete = new HashSet<>();
+
+        void log(State staged)
+        {
+            update.removeAll(staged.obsolete);
+            update.removeAll(staged.update);
+            update.addAll(staged.update);
+            obsolete.addAll(staged.obsolete);
+        }
+
+        boolean contains(SSTableReader reader)
+        {
+            return update.contains(reader) || obsolete.contains(reader);
+        }
+
+        boolean isEmpty()
+        {
+            return update.isEmpty() && obsolete.isEmpty();
+        }
+
+        void clear()
+        {
+            update.clear();
+            obsolete.clear();
+        }
+    }
+
+    public final Tracker tracker;
+    private final OperationType operationType;
+    // the original readers this transaction was opened over, and that it guards
+    // (no other transactions may operate over these readers concurrently)
+    private final Set<SSTableReader> originals = new HashSet<>();
+    // the set of readers we've marked as compacting (only updated on creation and in checkpoint())
+    private final Set<SSTableReader> marked = new HashSet<>();
+    // the identity set of readers we've ever encountered; used to ensure we don't accidentally revisit the
+    // same version of a reader. potentially a dangerous property if there are reference counting bugs
+    // as they won't be caught until the transaction's lifespan is over.
+    private final Set<UniqueIdentifier> identities = Collections.newSetFromMap(new IdentityHashMap<UniqueIdentifier, Boolean>());
+
+    // changes that have been made visible
+    private final State logged = new State();
+    // changes that are pending
+    private final State staged = new State();
+
+    /**
+     * construct a Transaction for use in an offline operation
+     */
+    public static LifecycleTransaction offline(OperationType operationType, SSTableReader reader)
+    {
+        return offline(operationType, singleton(reader));
+    }
+
+    /**
+     * construct a Transaction for use in an offline operation
+     */
+    public static LifecycleTransaction offline(OperationType operationType, Iterable<SSTableReader> readers)
+    {
+        // if offline, for simplicity we just use a dummy tracker
+        Tracker dummy = new Tracker(null, false);
+        dummy.addInitialSSTables(readers);
+        dummy.apply(updateCompacting(emptySet(), readers));
+        return new LifecycleTransaction(dummy, operationType, readers);
+    }
+
+    LifecycleTransaction(Tracker tracker, OperationType operationType, Iterable<SSTableReader> readers)
+    {
+        this.tracker = tracker;
+        this.operationType = operationType;
+        for (SSTableReader reader : readers)
+        {
+            originals.add(reader);
+            marked.add(reader);
+            identities.add(reader.instanceId);
+        }
+    }
+
+    public void doPrepare()
+    {
+        // note for future: in anticompaction two different operations use the same Transaction, and both prepareToCommit()
+        // separately: the second prepareToCommit is ignored as a "redundant" transition. since it is only a checkpoint
+        // (and these happen anyway) this is fine but if more logic gets inserted here than is performed in a checkpoint,
+        // it may break this use case, and care is needed
+        checkpoint();
+    }
+
+    /**
+     * point of no return: commit all changes, but leave all readers marked as compacting
+     */
+    public Throwable doCommit(Throwable accumulate)
+    {
+        assert staged.isEmpty() : "must be no actions introduced between prepareToCommit and a commit";
+
+        logger.debug("Committing update:{}, obsolete:{}", staged.update, staged.obsolete);
+
+        // this is now the point of no return; we cannot safely rollback, so we ignore exceptions until we're done
+        // we restore state by obsoleting our obsolete files, releasing our references to them, and updating our size
+        // and notification status for the obsolete and new files
+        accumulate = setupDeleteNotification(logged.update, tracker, accumulate);
+        accumulate = markObsolete(logged.obsolete, accumulate);
+        accumulate = tracker.updateSizeTracking(logged.obsolete, logged.update, accumulate);
+        accumulate = release(selfRefs(logged.obsolete), accumulate);
+        accumulate = tracker.notifySSTablesChanged(originals, logged.update, operationType, accumulate);
+        return accumulate;
+    }
+
+    /**
+     * undo all of the changes made by this transaction, resetting the state to its original form
+     */
+    public Throwable doAbort(Throwable accumulate)
+    {
+        if (logger.isDebugEnabled())
+            logger.debug("Aborting transaction over {}, with ({},{}) logged and ({},{}) staged", originals, logged.update, logged.obsolete, staged.update, staged.obsolete);
+
+        if (logged.isEmpty() && staged.isEmpty())
+            return accumulate;
+
+        // mark obsolete all readers that are not versions of those present in the original set
+        Iterable<SSTableReader> obsolete = filterOut(concatUniq(staged.update, logged.update), originals);
+        logger.debug("Obsoleting {}", obsolete);
+        accumulate = markObsolete(obsolete, accumulate);
+
+        // replace all updated readers with a version restored to its original state
+        accumulate = tracker.apply(updateLiveSet(logged.update, restoreUpdatedOriginals()), accumulate);
+        // setReplaced immediately preceding versions that have not been obsoleted
+        accumulate = setReplaced(logged.update, accumulate);
+        // we have replaced all of logged.update and never made visible staged.update,
+        // and the files we have logged as obsolete we clone fresh versions of, so they are no longer needed either
+        // any _staged_ obsoletes should either be in staged.update already, and dealt with there,
+        // or is still in its original form (so left as is); in either case no extra action is needed
+        accumulate = release(selfRefs(concat(staged.update, logged.update, logged.obsolete)), accumulate);
+        logged.clear();
+        staged.clear();
+        return accumulate;
+    }
+
+    @Override
+    protected Throwable doPostCleanup(Throwable accumulate)
+    {
+        return unmarkCompacting(marked, accumulate);
+    }
+
+    public void permitRedundantTransitions()
+    {
+        super.permitRedundantTransitions();
+    }
+
+    /**
+     * call when a consistent batch of changes is ready to be made atomically visible
+     * these will be exposed in the Tracker atomically, or an exception will be thrown; in this case
+     * the transaction should be rolled back
+     */
+    public void checkpoint()
+    {
+        maybeFail(checkpoint(null));
+    }
+    private Throwable checkpoint(Throwable accumulate)
+    {
+        if (logger.isDebugEnabled())
+            logger.debug("Checkpointing update:{}, obsolete:{}", staged.update, staged.obsolete);
+
+        if (staged.isEmpty())
+            return accumulate;
+
+        Set<SSTableReader> toUpdate = toUpdate();
+        Set<SSTableReader> fresh = copyOf(fresh());
+
+        // check the current versions of the readers we're replacing haven't somehow been replaced by someone else
+        checkNotReplaced(filterIn(toUpdate, staged.update));
+
+        // ensure any new readers are in the compacting set, since we aren't done with them yet
+        // and don't want anyone else messing with them
+        // apply atomically along with updating the live set of readers
+        tracker.apply(compose(updateCompacting(emptySet(), fresh),
+                              updateLiveSet(toUpdate, staged.update)));
+
+        // log the staged changes and our newly marked readers
+        marked.addAll(fresh);
+        logged.log(staged);
+
+        // setup our tracker, and mark our prior versions replaced, also releasing our references to them
+        // we do not replace/release obsoleted readers, since we may need to restore them on rollback
+        accumulate = setReplaced(filterOut(toUpdate, staged.obsolete), accumulate);
+        accumulate = release(selfRefs(filterOut(toUpdate, staged.obsolete)), accumulate);
+
+        staged.clear();
+        return accumulate;
+    }
+
+    /**
+     * update a reader: if !original, this is a reader that is being introduced by this transaction;
+     * otherwise it must be in the originals() set, i.e. a reader guarded by this transaction
+     */
+    public void update(SSTableReader reader, boolean original)
+    {
+        assert !staged.update.contains(reader) : "each reader may only be updated once per checkpoint: " + reader;
+        assert !identities.contains(reader.instanceId) : "each reader instance may only be provided as an update once: " + reader;
+        // check it isn't obsolete, and that it matches the original flag
+        assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not update a reader that has been obsoleted";
+        assert original == originals.contains(reader) : String.format("the 'original' indicator was incorrect (%s provided): %s", original, reader);
+        staged.update.add(reader);
+        identities.add(reader.instanceId);
+        reader.setupKeyCache();
+    }
+
+    /**
+     * mark this reader as for obsoletion. this does not actually obsolete the reader until commit() is called,
+     * but on checkpoint() the reader will be removed from the live set
+     */
+    public void obsolete(SSTableReader reader)
+    {
+        logger.debug("Staging for obsolescence {}", reader);
+        // check this is: a reader guarded by the transaction, an instance we have already worked with
+        // and that we haven't already obsoleted it, nor do we have other changes staged for it
+        assert identities.contains(reader.instanceId) : "only reader instances that have previously been provided may be obsoleted: " + reader;
+        assert originals.contains(reader) : "only readers in the 'original' set may be obsoleted: " + reader + " vs " + originals;
+        assert !(logged.obsolete.contains(reader) || staged.obsolete.contains(reader)) : "may not obsolete a reader that has already been obsoleted: " + reader;
+        assert !staged.update.contains(reader) : "may not obsolete a reader that has a staged update (must checkpoint first): " + reader;
+        assert current(reader) == reader : "may only obsolete the latest version of the reader: " + reader;
+        staged.obsolete.add(reader);
+    }
+
+    /**
+     * obsolete every file in the original transaction
+     */
+    public void obsoleteOriginals()
+    {
+        logger.debug("Staging for obsolescence {}", originals);
+        // if we're obsoleting, we should have no staged updates for the original files
+        assert Iterables.isEmpty(filterIn(staged.update, originals)) : staged.update;
+
+        // stage obsoletes for any currently visible versions of any original readers
+        Iterables.addAll(staged.obsolete, filterIn(current(), originals));
+    }
+
+    /**
+     * return the readers we're replacing in checkpoint(), i.e. the currently visible version of those in staged
+     */
+    private Set<SSTableReader> toUpdate()
+    {
+        return copyOf(filterIn(current(), staged.obsolete, staged.update));
+    }
+
+    /**
+     * new readers that haven't appeared previously (either in the original set or the logged updates)
+     */
+    private Iterable<SSTableReader> fresh()
+    {
+        return filterOut(staged.update,
+                         originals, logged.update);
+    }
+
+    /**
+     * returns the currently visible readers managed by this transaction
+     */
+    public Iterable<SSTableReader> current()
+    {
+        // i.e., those that are updates that have been logged (made visible),
+        // and any original readers that have neither been obsoleted nor updated
+        return concat(logged.update, filterOut(originals, logged.update, logged.obsolete));
+    }
+
+    /**
+     * update the current replacement of any original reader back to its original start
+     */
+    private List<SSTableReader> restoreUpdatedOriginals()
+    {
+        Iterable<SSTableReader> torestore = filterIn(originals, logged.update, logged.obsolete);
+        return ImmutableList.copyOf(transform(torestore,
+                                              new Function<SSTableReader, SSTableReader>()
+                                              {
+                                                  public SSTableReader apply(SSTableReader reader)
+                                                  {
+                                                      return current(reader).cloneWithNewStart(reader.first, null);
+                                                  }
+                                              }));
+    }
+
+    /**
+     * the set of readers guarded by this transaction _in their original instance/state_
+     * call current(SSTableReader) on any reader in this set to get the latest instance
+     */
+    public Set<SSTableReader> originals()
+    {
+        return Collections.unmodifiableSet(originals);
+    }
+
+    /**
+     * indicates if the reader has been marked for obsoletion
+     */
+    public boolean isObsolete(SSTableReader reader)
+    {
+        return logged.obsolete.contains(reader) || staged.obsolete.contains(reader);
+    }
+
+    /**
+     * return the current version of the provided reader, whether or not it is visible or staged;
+     * i.e. returns the first version present by testing staged, logged and originals in order.
+     */
+    public SSTableReader current(SSTableReader reader)
+    {
+        Set<SSTableReader> container;
+        if (staged.contains(reader))
+            container = staged.update.contains(reader) ? staged.update : staged.obsolete;
+        else if (logged.contains(reader))
+            container = logged.update.contains(reader) ? logged.update : logged.obsolete;
+        else if (originals.contains(reader))
+            container = originals;
+        else throw new AssertionError();
+        return select(reader, container);
+    }
+
+    /**
+     * remove the reader from the set we're modifying
+     */
+    public void cancel(SSTableReader cancel)
+    {
+        logger.debug("Cancelling {} from transaction", cancel);
+        assert originals.contains(cancel) : "may only cancel a reader in the 'original' set: " + cancel + " vs " + originals;
+        assert !(staged.contains(cancel) || logged.contains(cancel)) : "may only cancel a reader that has not been updated or obsoleted in this transaction: " + cancel;
+        originals.remove(cancel);
+        marked.remove(cancel);
+        maybeFail(unmarkCompacting(singleton(cancel), null));
+    }
+
+    /**
+     * remove the readers from the set we're modifying
+     */
+    public void cancel(Iterable<SSTableReader> cancels)
+    {
+        for (SSTableReader cancel : cancels)
+            cancel(cancel);
+    }
+
+    /**
+     * remove the provided readers from this Transaction, and return a new Transaction to manage them
+     * only permitted to be called if the current Transaction has never been used
+     */
+    public LifecycleTransaction split(Collection<SSTableReader> readers)
+    {
+        logger.debug("Splitting {} into new transaction", readers);
+        checkUnused();
+        for (SSTableReader reader : readers)
+            assert identities.contains(reader.instanceId) : "may only split the same reader instance the transaction was opened with: " + reader;
+
+        for (SSTableReader reader : readers)
+        {
+            identities.remove(reader.instanceId);
+            originals.remove(reader);
+            marked.remove(reader);
+        }
+        return new LifecycleTransaction(tracker, operationType, readers);
+    }
+
+    /**
+     * check this transaction has never been used
+     */
+    private void checkUnused()
+    {
+        assert logged.isEmpty();
+        assert staged.isEmpty();
+        assert identities.size() == originals.size();
+        assert originals.size() == marked.size();
+    }
+
+    private Throwable unmarkCompacting(Set<SSTableReader> unmark, Throwable accumulate)
+    {
+        accumulate = tracker.apply(updateCompacting(unmark, emptySet()), accumulate);
+        // when the CFS is invalidated, it will call unreferenceSSTables().  However, unreferenceSSTables only deals
+        // with sstables that aren't currently being compacted.  If there are ongoing compactions that finish or are
+        // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
+        accumulate = tracker.dropSSTablesIfInvalid(accumulate);
+        return accumulate;
+    }
+
+    // convenience method for callers that know only one sstable is involved in the transaction
+    public SSTableReader onlyOne()
+    {
+        assert originals.size() == 1;
+        return getFirst(originals, null);
+    }
+
+    // a class representing the current state of the reader within this transaction, encoding the actions both logged
+    // and pending, and the reader instances that are visible now, and will be after the next checkpoint (with null
+    // indicating either obsolescence, or that the reader does not occur in the transaction; which is defined
+    // by the corresponding Action)
+    @VisibleForTesting
+    public static class ReaderState
+    {
+        public static enum Action
+        {
+            UPDATED, OBSOLETED, NONE;
+            public static Action get(boolean updated, boolean obsoleted)
+            {
+                assert !(updated && obsoleted);
+                return updated ? UPDATED : obsoleted ? OBSOLETED : NONE;
+            }
+        }
+
+        final Action staged;
+        final Action logged;
+        final SSTableReader nextVisible;
+        final SSTableReader currentlyVisible;
+        final boolean original;
+
+        public ReaderState(Action logged, Action staged, SSTableReader currentlyVisible, SSTableReader nextVisible, boolean original)
+        {
+            this.staged = staged;
+            this.logged = logged;
+            this.currentlyVisible = currentlyVisible;
+            this.nextVisible = nextVisible;
+            this.original = original;
+        }
+
+        public boolean equals(Object that)
+        {
+            return that instanceof ReaderState && equals((ReaderState) that);
+        }
+
+        public boolean equals(ReaderState that)
+        {
+            return this.staged == that.staged && this.logged == that.logged && this.original == that.original
+                && this.currentlyVisible == that.currentlyVisible && this.nextVisible == that.nextVisible;
+        }
+
+        public String toString()
+        {
+            return String.format("[logged=%s staged=%s original=%s]", logged, staged, original);
+        }
+
+        public static SSTableReader visible(SSTableReader reader, Predicate<SSTableReader> obsolete, Collection<SSTableReader> ... selectFrom)
+        {
+            return obsolete.apply(reader) ? null : selectFirst(reader, selectFrom);
+        }
+    }
+
+    @VisibleForTesting
+    public ReaderState state(SSTableReader reader)
+    {
+        SSTableReader currentlyVisible = ReaderState.visible(reader, in(logged.obsolete), logged.update, originals);
+        SSTableReader nextVisible = ReaderState.visible(reader, orIn(staged.obsolete, logged.obsolete), staged.update, logged.update, originals);
+        return new ReaderState(ReaderState.Action.get(logged.update.contains(reader), logged.obsolete.contains(reader)),
+                               ReaderState.Action.get(staged.update.contains(reader), staged.obsolete.contains(reader)),
+                               currentlyVisible, nextVisible, originals.contains(reader)
+        );
+    }
+
+    public String toString()
+    {
+        return originals.toString();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
new file mode 100644
index 0000000..ff2abcb
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/SSTableIntervalTree.java
@@ -0,0 +1,40 @@
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+import org.apache.cassandra.utils.IntervalTree;
+
+public class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
+{
+    private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
+
+    SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
+    {
+        super(intervals);
+    }
+
+    public static SSTableIntervalTree empty()
+    {
+        return EMPTY;
+    }
+
+    public static SSTableIntervalTree build(Iterable<SSTableReader> sstables)
+    {
+        return new SSTableIntervalTree(buildIntervals(sstables));
+    }
+
+    public static List<Interval<RowPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
+    {
+        List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
+        for (SSTableReader sstable : sstables)
+            intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
+        return intervals;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
new file mode 100644
index 0000000..50f567f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -0,0 +1,468 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.io.File;
+import java.util.*;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Function;
+import com.google.common.base.Predicate;
+import com.google.common.base.Predicates;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.metrics.StorageMetrics;
+import org.apache.cassandra.notifications.*;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import static com.google.common.base.Predicates.and;
+import static com.google.common.collect.ImmutableSet.copyOf;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.*;
+import static org.apache.cassandra.db.lifecycle.View.permitCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateCompacting;
+import static org.apache.cassandra.db.lifecycle.View.updateLiveSet;
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+import static org.apache.cassandra.utils.concurrent.Refs.release;
+import static org.apache.cassandra.utils.concurrent.Refs.selfRefs;
+
+public class Tracker
+{
+    private static final Logger logger = LoggerFactory.getLogger(Tracker.class);
+
+    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
+    public final ColumnFamilyStore cfstore;
+    final AtomicReference<View> view;
+    public final boolean loadsstables;
+
+    public Tracker(ColumnFamilyStore cfstore, boolean loadsstables)
+    {
+        this.cfstore = cfstore;
+        this.view = new AtomicReference<>();
+        this.loadsstables = loadsstables;
+        this.reset();
+    }
+
+    public LifecycleTransaction tryModify(SSTableReader sstable, OperationType operationType)
+    {
+        return tryModify(singleton(sstable), operationType);
+    }
+
+    /**
+     * @return a Transaction over the provided sstables if we are able to mark the given @param sstables as compacted, before anyone else
+     */
+    public LifecycleTransaction tryModify(Iterable<SSTableReader> sstables, OperationType operationType)
+    {
+        if (Iterables.isEmpty(sstables))
+            return new LifecycleTransaction(this, operationType, sstables);
+        if (null == apply(permitCompacting(sstables), updateCompacting(emptySet(), sstables)))
+            return null;
+        return new LifecycleTransaction(this, operationType, sstables);
+    }
+
+
+    // METHODS FOR ATOMICALLY MODIFYING THE VIEW
+
+    Pair<View, View> apply(Function<View, View> function)
+    {
+        return apply(Predicates.<View>alwaysTrue(), function);
+    }
+
+    Throwable apply(Function<View, View> function, Throwable accumulate)
+    {
+        try
+        {
+            apply(function);
+        }
+        catch (Throwable t)
+        {
+            accumulate = merge(accumulate, t);
+        }
+        return accumulate;
+    }
+
+    /**
+     * atomically tests permit against the view and applies function to it, if permit yields true, returning the original;
+     * otherwise the method aborts, returning null
+     */
+    Pair<View, View> apply(Predicate<View> permit, Function<View, View> function)
+    {
+        while (true)
+        {
+            View cur = view.get();
+            if (!permit.apply(cur))
+                return null;
+            View updated = function.apply(cur);
+            if (view.compareAndSet(cur, updated))
+                return Pair.create(cur, updated);
+        }
+    }
+
+    Throwable updateSizeTracking(Iterable<SSTableReader> oldSSTables, Iterable<SSTableReader> newSSTables, Throwable accumulate)
+    {
+        if (cfstore == null)
+            return accumulate;
+
+        long add = 0;
+        for (SSTableReader sstable : newSSTables)
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("adding {} to list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            try
+            {
+                add += sstable.bytesOnDisk();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        long subtract = 0;
+        for (SSTableReader sstable : oldSSTables)
+        {
+            if (logger.isDebugEnabled())
+                logger.debug("removing {} from list of files tracked for {}.{}", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name);
+            try
+            {
+                subtract += sstable.bytesOnDisk();
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        StorageMetrics.load.inc(add - subtract);
+        cfstore.metric.liveDiskSpaceUsed.inc(add - subtract);
+        // we don't subtract from total until the sstable is deleted
+        cfstore.metric.totalDiskSpaceUsed.inc(add);
+        return accumulate;
+    }
+
+    // SETUP / CLEANUP
+
+    public void addInitialSSTables(Iterable<SSTableReader> sstables)
+    {
+        maybeFail(setupDeleteNotification(sstables, this, null));
+        apply(updateLiveSet(emptySet(), sstables));
+        maybeFail(updateSizeTracking(emptySet(), sstables, null));
+        // no notifications or backup necessary
+    }
+
+    public void addSSTables(Iterable<SSTableReader> sstables)
+    {
+        addInitialSSTables(sstables);
+        for (SSTableReader sstable : sstables)
+        {
+            maybeIncrementallyBackup(sstable);
+            notifyAdded(sstable);
+        }
+    }
+
+    /** (Re)initializes the tracker, purging all references. */
+    @VisibleForTesting
+    public void reset()
+    {
+        view.set(new View(
+                         cfstore != null ? ImmutableList.of(new Memtable(cfstore)) : Collections.<Memtable>emptyList(),
+                         ImmutableList.<Memtable>of(),
+                         Collections.<SSTableReader, SSTableReader>emptyMap(),
+                         Collections.<SSTableReader>emptySet(),
+                         SSTableIntervalTree.empty()));
+    }
+
+    public Throwable dropSSTablesIfInvalid(Throwable accumulate)
+    {
+        if (cfstore != null && !cfstore.isValid())
+            accumulate = dropSSTables(accumulate);
+        return accumulate;
+    }
+
+    public void dropSSTables()
+    {
+        maybeFail(dropSSTables(null));
+    }
+
+    public Throwable dropSSTables(Throwable accumulate)
+    {
+        return dropSSTables(Predicates.<SSTableReader>alwaysTrue(), OperationType.UNKNOWN, accumulate);
+    }
+
+    /**
+     * removes all sstables that are not busy compacting.
+     */
+    public Throwable dropSSTables(final Predicate<SSTableReader> remove, OperationType operationType, Throwable accumulate)
+    {
+        Pair<View, View> result = apply(new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                Set<SSTableReader> toremove = copyOf(filter(view.sstables, and(remove, notIn(view.compacting))));
+                return updateLiveSet(toremove, emptySet()).apply(view);
+            }
+        });
+
+        Set<SSTableReader> removed = Sets.difference(result.left.sstables, result.right.sstables);
+        assert Iterables.all(removed, remove);
+
+        if (!removed.isEmpty())
+        {
+            // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
+            accumulate = notifySSTablesChanged(removed, Collections.<SSTableReader>emptySet(), operationType, accumulate);
+            accumulate = updateSizeTracking(removed, emptySet(), accumulate);
+            accumulate = markObsolete(removed, accumulate);
+            accumulate = release(selfRefs(removed), accumulate);
+        }
+        return accumulate;
+    }
+
+    /**
+     * Removes every SSTable in the directory from the Tracker's view.
+     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
+     */
+    public void removeUnreadableSSTables(final File directory)
+    {
+        maybeFail(dropSSTables(new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader reader)
+            {
+                return reader.descriptor.directory.equals(directory);
+            }
+        }, OperationType.UNKNOWN, null));
+    }
+
+
+
+    // FLUSHING
+
+    /**
+     * get the Memtable that the ordered writeOp should be directed to
+     */
+    public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
+    {
+        // since any new memtables appended to the list after we fetch it will be for operations started
+        // after us, we can safely assume that we will always find the memtable that 'accepts' us;
+        // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
+
+        // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
+        // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
+        // assign operations to a memtable that was retired/queued before we started)
+        for (Memtable memtable : view.get().liveMemtables)
+        {
+            if (memtable.accepts(opGroup, replayPosition))
+                return memtable;
+        }
+        throw new AssertionError(view.get().liveMemtables.toString());
+    }
+
+    /**
+     * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
+     * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
+     * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
+     * must be followed by discarding(m), they cannot be interleaved.
+     *
+     * @return the previously active memtable
+     */
+    public Memtable switchMemtable(boolean truncating)
+    {
+        Memtable newMemtable = new Memtable(cfstore);
+        Pair<View, View> result = apply(View.switchMemtable(newMemtable));
+        if (truncating)
+            notifyRenewed(newMemtable);
+
+        return result.left.getCurrentMemtable();
+    }
+
+    public void markFlushing(Memtable memtable)
+    {
+        apply(View.markFlushing(memtable));
+    }
+
+    public void replaceFlushed(Memtable memtable, SSTableReader sstable)
+    {
+        if (sstable == null)
+        {
+            // sstable may be null if we flushed batchlog and nothing needed to be retained
+            // if it's null, we don't care what state the cfstore is in, we just replace it and continue
+            apply(View.replaceFlushed(memtable, null));
+            return;
+        }
+
+        sstable.setupDeleteNotification(this);
+        sstable.setupKeyCache();
+        // back up before creating a new Snapshot (which makes the new one eligible for compaction)
+        maybeIncrementallyBackup(sstable);
+
+        apply(View.replaceFlushed(memtable, sstable));
+
+        Throwable fail;
+        fail = updateSizeTracking(emptySet(), singleton(sstable), null);
+        // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+        fail = notifyAdded(sstable, fail);
+
+        if (cfstore != null && !cfstore.isValid())
+            dropSSTables();
+
+        maybeFail(fail);
+    }
+
+
+
+    // MISCELLANEOUS public utility calls
+
+    public Set<SSTableReader> getSSTables()
+    {
+        return view.get().sstables;
+    }
+
+    public Set<SSTableReader> getCompacting()
+    {
+        return view.get().compacting;
+    }
+
+    public Set<SSTableReader> getUncompacting()
+    {
+        return view.get().nonCompactingSStables();
+    }
+
+    public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+    {
+        return view.get().getUncompacting(candidates);
+    }
+
+    public void maybeIncrementallyBackup(final SSTableReader sstable)
+    {
+        if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
+            return;
+
+        File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
+        sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
+    }
+
+    public void spaceReclaimed(long size)
+    {
+        if (cfstore != null)
+            cfstore.metric.totalDiskSpaceUsed.dec(size);
+    }
+
+
+
+    // NOTIFICATION
+
+    Throwable notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType, Throwable accumulate)
+    {
+        INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
+        for (INotificationConsumer subscriber : subscribers)
+        {
+            try
+            {
+                subscriber.handleNotification(notification, this);
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    Throwable notifyAdded(SSTableReader added, Throwable accumulate)
+    {
+        INotification notification = new SSTableAddedNotification(added);
+        for (INotificationConsumer subscriber : subscribers)
+        {
+            try
+            {
+                subscriber.handleNotification(notification, this);
+            }
+            catch (Throwable t)
+            {
+                accumulate = merge(accumulate, t);
+            }
+        }
+        return accumulate;
+    }
+
+    public void notifyAdded(SSTableReader added)
+    {
+        maybeFail(notifyAdded(added, null));
+    }
+
+    public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
+    {
+        INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyDeleting(SSTableReader deleting)
+    {
+        INotification notification = new SSTableDeletingNotification(deleting);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyRenewed(Memtable renewed)
+    {
+        INotification notification = new MemtableRenewedNotification(renewed);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void notifyTruncated(long truncatedAt)
+    {
+        INotification notification = new TruncationNotification(truncatedAt);
+        for (INotificationConsumer subscriber : subscribers)
+            subscriber.handleNotification(notification, this);
+    }
+
+    public void subscribe(INotificationConsumer consumer)
+    {
+        subscribers.add(consumer);
+    }
+
+    public void unsubscribe(INotificationConsumer consumer)
+    {
+        subscribers.remove(consumer);
+    }
+
+    private static Set<SSTableReader> emptySet()
+    {
+        return Collections.emptySet();
+    }
+
+    public View getView()
+    {
+        return view.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/lifecycle/View.java b/src/java/org/apache/cassandra/db/lifecycle/View.java
new file mode 100644
index 0000000..0d1100b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db.lifecycle;
+
+import java.util.*;
+
+import com.google.common.base.Function;
+import com.google.common.base.Functions;
+import com.google.common.base.Predicate;
+import com.google.common.collect.*;
+
+import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.utils.Interval;
+
+import static com.google.common.base.Predicates.equalTo;
+import static com.google.common.base.Predicates.not;
+import static com.google.common.collect.ImmutableList.copyOf;
+import static com.google.common.collect.ImmutableList.of;
+import static com.google.common.collect.Iterables.all;
+import static com.google.common.collect.Iterables.concat;
+import static com.google.common.collect.Iterables.filter;
+import static java.util.Collections.singleton;
+import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
+import static org.apache.cassandra.db.lifecycle.Helpers.replace;
+
+/**
+ * An immutable structure holding the current memtable, the memtables pending
+ * flush, the sstables for a column family, and the sstables that are active
+ * in compaction (a subset of the sstables).
+ *
+ * Modifications to instances are all performed via a Function produced by the static methods in this class.
+ * These are composed as necessary and provided to the Tracker.apply() methods, which atomically reject or
+ * accept and apply the changes to the View.
+ *
+ */
+public class View
+{
+    /**
+     * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
+     * and the new replacement memtable, until all outstanding write operations on the old table complete.
+     * The last item in the list is always the "current" memtable.
+     */
+    public final List<Memtable> liveMemtables;
+    /**
+     * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
+     * flushed. In chronologically ascending order.
+     */
+    public final List<Memtable> flushingMemtables;
+    public final Set<SSTableReader> compacting;
+    public final Set<SSTableReader> sstables;
+    // we use a Map here so that we can easily perform identity checks as well as equality checks.
+    // When marking compacting, we now  indicate if we expect the sstables to be present (by default we do),
+    // and we then check that not only are they all present in the live set, but that the exact instance present is
+    // the one we made our decision to compact against.
+    public final Map<SSTableReader, SSTableReader> sstablesMap;
+
+    public final SSTableIntervalTree intervalTree;
+
+    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, SSTableIntervalTree intervalTree)
+    {
+        assert liveMemtables != null;
+        assert flushingMemtables != null;
+        assert sstables != null;
+        assert compacting != null;
+        assert intervalTree != null;
+
+        this.liveMemtables = liveMemtables;
+        this.flushingMemtables = flushingMemtables;
+
+        this.sstablesMap = sstables;
+        this.sstables = sstablesMap.keySet();
+        this.compacting = compacting;
+        this.intervalTree = intervalTree;
+    }
+
+    public Memtable getCurrentMemtable()
+    {
+        return liveMemtables.get(liveMemtables.size() - 1);
+    }
+
+    /**
+     * @return the active memtable and all the memtables that are pending flush.
+     */
+    public Iterable<Memtable> getAllMemtables()
+    {
+        return concat(flushingMemtables, liveMemtables);
+    }
+
+    public Sets.SetView<SSTableReader> nonCompactingSStables()
+    {
+        return Sets.difference(sstables, compacting);
+    }
+
+    public Iterable<SSTableReader> getUncompacting(Iterable<SSTableReader> candidates)
+    {
+        return filter(candidates, new Predicate<SSTableReader>()
+        {
+            public boolean apply(SSTableReader sstable)
+            {
+                return !compacting.contains(sstable);
+            }
+        });
+    }
+
+    @Override
+    public String toString()
+    {
+        return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
+    }
+
+    public List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds)
+    {
+        if (intervalTree.isEmpty())
+            return Collections.emptyList();
+        RowPosition stopInTree = rowBounds.right.isMinimum() ? intervalTree.max() : rowBounds.right;
+        return intervalTree.search(Interval.<RowPosition, SSTableReader>create(rowBounds.left, stopInTree));
+    }
+
+    // METHODS TO CONSTRUCT FUNCTIONS FOR MODIFYING A VIEW:
+
+    // return a function to un/mark the provided readers compacting in a view
+    static Function<View, View> updateCompacting(final Set<SSTableReader> unmark, final Iterable<SSTableReader> mark)
+    {
+        if (unmark.isEmpty() && Iterables.isEmpty(mark))
+            return Functions.identity();
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                assert all(mark, Helpers.idIn(view.sstablesMap));
+                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
+                                replace(view.compacting, unmark, mark),
+                                view.intervalTree);
+            }
+        };
+    }
+
+    // construct a predicate to reject views that do not permit us to mark these readers compacting;
+    // i.e. one of them is either already compacting, has been compacted, or has been replaced
+    static Predicate<View> permitCompacting(final Iterable<SSTableReader> readers)
+    {
+        return new Predicate<View>()
+        {
+            public boolean apply(View view)
+            {
+                for (SSTableReader reader : readers)
+                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted())
+                        return false;
+                return true;
+            }
+        };
+    }
+
+    // construct a function to change the liveset in a Snapshot
+    static Function<View, View> updateLiveSet(final Set<SSTableReader> remove, final Iterable<SSTableReader> add)
+    {
+        if (remove.isEmpty() && Iterables.isEmpty(add))
+            return Functions.identity();
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
+                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compacting,
+                                SSTableIntervalTree.build(sstableMap.keySet()));
+            }
+        };
+    }
+
+    // called prior to initiating flush: add newMemtable to liveMemtables, making it the latest memtable
+    static Function<View, View> switchMemtable(final Memtable newMemtable)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> newLive = ImmutableList.<Memtable>builder().addAll(view.liveMemtables).add(newMemtable).build();
+                assert newLive.size() == view.liveMemtables.size() + 1;
+                return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compacting, view.intervalTree);
+            }
+        };
+    }
+
+    // called before flush: move toFlush from liveMemtables to flushingMemtables
+    static Function<View, View> markFlushing(final Memtable toFlush)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> live = view.liveMemtables, flushing = view.flushingMemtables;
+                List<Memtable> newLive = copyOf(filter(live, not(equalTo(toFlush))));
+                List<Memtable> newFlushing = copyOf(concat(filter(flushing, lessThan(toFlush)),
+                                                           of(toFlush),
+                                                           filter(flushing, not(lessThan(toFlush)))));
+                assert newLive.size() == live.size() - 1;
+                assert newFlushing.size() == flushing.size() + 1;
+                return new View(newLive, newFlushing, view.sstablesMap, view.compacting, view.intervalTree);
+            }
+        };
+    }
+
+    // called after flush: removes memtable from flushingMemtables, and inserts flushed into the live sstable set
+    static Function<View, View> replaceFlushed(final Memtable memtable, final SSTableReader flushed)
+    {
+        return new Function<View, View>()
+        {
+            public View apply(View view)
+            {
+                List<Memtable> flushingMemtables = copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
+                assert flushingMemtables.size() == view.flushingMemtables.size() - 1;
+
+                if (flushed == null)
+                    return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
+                                    view.compacting, view.intervalTree);
+
+                Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), singleton(flushed));
+                return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compacting,
+                                SSTableIntervalTree.build(sstableMap.keySet()));
+            }
+        };
+    }
+
+    private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
+    {
+        return new Predicate<T>()
+        {
+            public boolean apply(T t)
+            {
+                return t.compareTo(lessThan) < 0;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
index a6c7a8b..c994a3d 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressionMetadata.java
@@ -403,7 +403,7 @@ public class CompressionMetadata
             count = chunkIndex;
         }
 
-        protected Throwable doCleanup(Throwable failed)
+        protected Throwable doPreCleanup(Throwable failed)
         {
             return offsets.close(failed);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
index 81e487c..9bfbc99 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryManager.java
@@ -19,24 +19,15 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.IOException;
 import java.lang.management.ManagementFactory;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multimap;
-import com.google.common.collect.Sets;
+import com.google.common.collect.*;
+
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -44,8 +35,10 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
 
@@ -209,22 +202,25 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
      * Returns a Pair of all compacting and non-compacting sstables.  Non-compacting sstables will be marked as
      * compacting.
      */
-    private Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> getCompactingAndNonCompactingSSTables()
+    private Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> getCompactingAndNonCompactingSSTables()
     {
         List<SSTableReader> allCompacting = new ArrayList<>();
-        Multimap<DataTracker, SSTableReader> allNonCompacting = HashMultimap.create();
+        Map<UUID, LifecycleTransaction> allNonCompacting = new HashMap<>();
         for (Keyspace ks : Keyspace.all())
         {
             for (ColumnFamilyStore cfStore: ks.getColumnFamilyStores())
             {
                 Set<SSTableReader> nonCompacting, allSSTables;
+                LifecycleTransaction txn = null;
                 do
                 {
-                    allSSTables = cfStore.getDataTracker().getSSTables();
-                    nonCompacting = Sets.newHashSet(cfStore.getDataTracker().getUncompactingSSTables(allSSTables));
+                    View view = cfStore.getTracker().getView();
+                    allSSTables = view.sstables;
+                    nonCompacting = ImmutableSet.copyOf(view.getUncompacting(allSSTables));
                 }
-                while (!(nonCompacting.isEmpty() || cfStore.getDataTracker().markCompacting(nonCompacting)));
-                allNonCompacting.putAll(cfStore.getDataTracker(), nonCompacting);
+                while (null == (txn = cfStore.getTracker().tryModify(nonCompacting, OperationType.UNKNOWN)));
+
+                allNonCompacting.put(cfStore.metadata.cfId, txn);
                 allCompacting.addAll(Sets.difference(allSSTables, nonCompacting));
             }
         }
@@ -233,50 +229,57 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
 
     public void redistributeSummaries() throws IOException
     {
-        Pair<List<SSTableReader>, Multimap<DataTracker, SSTableReader>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
+        Pair<List<SSTableReader>, Map<UUID, LifecycleTransaction>> compactingAndNonCompacting = getCompactingAndNonCompactingSSTables();
         try
         {
-            redistributeSummaries(compactingAndNonCompacting.left, Lists.newArrayList(compactingAndNonCompacting.right.values()), this.memoryPoolBytes);
+            redistributeSummaries(compactingAndNonCompacting.left, compactingAndNonCompacting.right, this.memoryPoolBytes);
         }
         finally
         {
-            for(DataTracker tracker : compactingAndNonCompacting.right.keySet())
-                tracker.unmarkCompacting(compactingAndNonCompacting.right.get(tracker));
+            for (LifecycleTransaction modifier : compactingAndNonCompacting.right.values())
+                modifier.close();
         }
     }
 
     /**
      * Attempts to fairly distribute a fixed pool of memory for index summaries across a set of SSTables based on
      * their recent read rates.
-     * @param nonCompacting a list of sstables to share the memory pool across
+     * @param transactions containing the sstables we are to redistribute the memory pool across
      * @param memoryPoolBytes a size (in bytes) that the total index summary space usage should stay close to or
      *                        under, if possible
      * @return a list of new SSTableReader instances
      */
     @VisibleForTesting
-    public static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting, List<SSTableReader> nonCompacting, long memoryPoolBytes) throws IOException
+    public static List<SSTableReader> redistributeSummaries(List<SSTableReader> compacting, Map<UUID, LifecycleTransaction> transactions, long memoryPoolBytes) throws IOException
     {
-        long total = 0;
-        for (SSTableReader sstable : Iterables.concat(compacting, nonCompacting))
-            total += sstable.getIndexSummaryOffHeapSize();
-
         List<SSTableReader> oldFormatSSTables = new ArrayList<>();
-        for (SSTableReader sstable : nonCompacting)
+        List<SSTableReader> redistribute = new ArrayList<>();
+        for (LifecycleTransaction txn : transactions.values())
         {
-            // We can't change the sampling level of sstables with the old format, because the serialization format
-            // doesn't include the sampling level.  Leave this one as it is.  (See CASSANDRA-8993 for details.)
-            logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
-            if (!sstable.descriptor.version.hasSamplingLevel())
-                oldFormatSSTables.add(sstable);
+            for (SSTableReader sstable : ImmutableList.copyOf(txn.originals()))
+            {
+                // We can't change the sampling level of sstables with the old format, because the serialization format
+                // doesn't include the sampling level.  Leave this one as it is.  (See CASSANDRA-8993 for details.)
+                logger.trace("SSTable {} cannot be re-sampled due to old sstable format", sstable);
+                if (!sstable.descriptor.version.hasSamplingLevel())
+                {
+                    oldFormatSSTables.add(sstable);
+                    txn.cancel(sstable);
+                }
+            }
+            redistribute.addAll(txn.originals());
         }
-        nonCompacting.removeAll(oldFormatSSTables);
+
+        long total = 0;
+        for (SSTableReader sstable : Iterables.concat(compacting, redistribute))
+            total += sstable.getIndexSummaryOffHeapSize();
 
         logger.debug("Beginning redistribution of index summaries for {} sstables with memory pool size {} MB; current spaced used is {} MB",
-                     nonCompacting.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
+                     redistribute.size(), memoryPoolBytes / 1024L / 1024L, total / 1024.0 / 1024.0);
 
-        final Map<SSTableReader, Double> readRates = new HashMap<>(nonCompacting.size());
+        final Map<SSTableReader, Double> readRates = new HashMap<>(redistribute.size());
         double totalReadsPerSec = 0.0;
-        for (SSTableReader sstable : nonCompacting)
+        for (SSTableReader sstable : redistribute)
         {
             if (sstable.getReadMeter() != null)
             {
@@ -288,7 +291,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         logger.trace("Total reads/sec across all sstables in index summary resize process: {}", totalReadsPerSec);
 
         // copy and sort by read rates (ascending)
-        List<SSTableReader> sstablesByHotness = new ArrayList<>(nonCompacting);
+        List<SSTableReader> sstablesByHotness = new ArrayList<>(redistribute);
         Collections.sort(sstablesByHotness, new ReadRateComparator(readRates));
 
         long remainingBytes = memoryPoolBytes;
@@ -297,7 +300,10 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
 
         logger.trace("Index summaries for compacting SSTables are using {} MB of space",
                      (memoryPoolBytes - remainingBytes) / 1024.0 / 1024.0);
-        List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, totalReadsPerSec, remainingBytes);
+        List<SSTableReader> newSSTables = adjustSamplingLevels(sstablesByHotness, transactions, totalReadsPerSec, remainingBytes);
+
+        for (LifecycleTransaction txn : transactions.values())
+            txn.finish();
 
         total = 0;
         for (SSTableReader sstable : Iterables.concat(compacting, oldFormatSSTables, newSSTables))
@@ -308,7 +314,7 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         return newSSTables;
     }
 
-    private static List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables,
+    private static List<SSTableReader> adjustSamplingLevels(List<SSTableReader> sstables, Map<UUID, LifecycleTransaction> transactions,
                                                             double totalReadsPerSec, long memoryPoolCapacity) throws IOException
     {
 
@@ -410,26 +416,16 @@ public class IndexSummaryManager implements IndexSummaryManagerMBean
         toDownsample.addAll(forceResample);
         toDownsample.addAll(toUpsample);
         toDownsample.addAll(forceUpsample);
-        Multimap<DataTracker, SSTableReader> replacedByTracker = HashMultimap.create();
-        Multimap<DataTracker, SSTableReader> replacementsByTracker = HashMultimap.create();
         for (ResampleEntry entry : toDownsample)
         {
             SSTableReader sstable = entry.sstable;
             logger.debug("Re-sampling index summary for {} from {}/{} to {}/{} of the original number of entries",
                          sstable, sstable.getIndexSummarySamplingLevel(), Downsampling.BASE_SAMPLING_LEVEL,
                          entry.newSamplingLevel, Downsampling.BASE_SAMPLING_LEVEL);
-            ColumnFamilyStore cfs = Keyspace.open(sstable.getKeyspaceName()).getColumnFamilyStore(sstable.getColumnFamilyName());
+            ColumnFamilyStore cfs = Keyspace.open(sstable.metadata.ksName).getColumnFamilyStore(sstable.metadata.cfId);
             SSTableReader replacement = sstable.cloneWithNewSummarySamplingLevel(cfs, entry.newSamplingLevel);
-            DataTracker tracker = cfs.getDataTracker();
-
-            replacedByTracker.put(tracker, sstable);
-            replacementsByTracker.put(tracker, replacement);
-        }
-
-        for (DataTracker tracker : replacedByTracker.keySet())
-        {
-            tracker.replaceWithNewInstances(replacedByTracker.get(tracker), replacementsByTracker.get(tracker));
-            newSSTables.addAll(replacementsByTracker.get(tracker));
+            newSSTables.add(replacement);
+            transactions.get(sstable.metadata.cfId).update(replacement, true);
         }
 
         return newSSTables;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
index db54557..cc837ba 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
@@ -23,15 +23,16 @@ import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Sets;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.db.DataTracker;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.lifecycle.Tracker;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.Blocker;
 
 public class SSTableDeletingTask implements Runnable
 {
@@ -42,11 +43,12 @@ public class SSTableDeletingTask implements Runnable
     // Additionally, we need to make sure to delete the data file first, so on restart the others
     // will be recognized as GCable.
     private static final Set<SSTableDeletingTask> failedTasks = new CopyOnWriteArraySet<>();
+    private static final Blocker blocker = new Blocker();
 
     private final SSTableReader referent;
     private final Descriptor desc;
     private final Set<Component> components;
-    private DataTracker tracker;
+    private Tracker tracker;
 
     /**
      * realDescriptor is the actual descriptor for the sstable, the descriptor inside
@@ -70,13 +72,18 @@ public class SSTableDeletingTask implements Runnable
         }
     }
 
-    public void setTracker(DataTracker tracker)
+    public void setTracker(Tracker tracker)
     {
         // the tracker is used only to notify listeners of deletion of the sstable;
         // since deletion of a non-final file is not really deletion of the sstable,
         // we don't want to notify the listeners in this event
-        if (desc.type == Descriptor.Type.FINAL)
-            this.tracker = tracker;
+        assert desc.type == Descriptor.Type.FINAL;
+        this.tracker = tracker;
+    }
+
+    public Tracker getTracker()
+    {
+        return tracker;
     }
 
     public void schedule()
@@ -86,6 +93,7 @@ public class SSTableDeletingTask implements Runnable
 
     public void run()
     {
+        blocker.ask();
         long size = referent.bytesOnDisk();
 
         if (tracker != null)
@@ -119,6 +127,7 @@ public class SSTableDeletingTask implements Runnable
     }
 
     /** for tests */
+    @VisibleForTesting
     public static void waitForDeletions()
     {
         Runnable runnable = new Runnable()
@@ -130,5 +139,11 @@ public class SSTableDeletingTask implements Runnable
 
         FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(runnable, 0, TimeUnit.MILLISECONDS));
     }
+
+    @VisibleForTesting
+    public static void pauseDeletions(boolean stop)
+    {
+        blocker.block(stop);
+    }
 }
 


[3/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Posted by be...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index a526ec9..8029075 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,33 +20,28 @@ package org.apache.cassandra.io.sstable;
 import java.util.*;
 
 import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.ImmutableList;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.compaction.AbstractCompactedRow;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.CLibrary;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.Refs;
 import org.apache.cassandra.utils.concurrent.Transactional;
 
-import static org.apache.cassandra.utils.Throwables.merge;
-
 /**
  * Wraps one or more writers as output for rewriting one or more readers: every sstable_preemptive_open_interval_in_mb
  * we look in the summary we're collecting for the latest writer for the penultimate key that we know to have been fully
  * flushed to the index file, and then double check that the key is fully present in the flushed data file.
- * Then we move the starts of each reader forwards to that point, replace them in the datatracker, and attach a runnable
+ * Then we move the starts of each reader forwards to that point, replace them in the Tracker, and attach a runnable
  * for on-close (i.e. when all references expire) that drops the page cache prior to that key position
  *
  * hard-links are created for each partially written sstable so that readers opened against them continue to work past
  * the rename of the temporary file, which is deleted once all readers against the hard-link have been closed.
- * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the DataTracker.
+ * If for any reason the writer is rolled over, we immediately rename and fully expose the completed file in the Tracker.
  *
  * On abort we restore the original lower bounds to the existing readers and delete any temporary files we had in progress,
  * but leave any hard-links in place for the readers we opened to cleanup when they're finished as we would had we finished
@@ -74,26 +69,19 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
         return preemptiveOpenInterval;
     }
 
-    private final DataTracker dataTracker;
     private final ColumnFamilyStore cfs;
 
     private final long maxAge;
     private long repairedAt = -1;
     // the set of final readers we will expose on commit
+    private final LifecycleTransaction transaction; // the readers we are rewriting (updated as they are replaced)
     private final List<SSTableReader> preparedForCommit = new ArrayList<>();
-    private final Set<SSTableReader> rewriting; // the readers we are rewriting (updated as they are replaced)
-    private final Map<Descriptor, DecoratedKey> originalStarts = new HashMap<>(); // the start key for each reader we are rewriting
     private final Map<Descriptor, Integer> fileDescriptors = new HashMap<>(); // the file descriptors for each reader descriptor we are rewriting
 
-    private SSTableReader currentlyOpenedEarly; // the reader for the most recent (re)opening of the target file
     private long currentlyOpenedEarlyAt; // the position (in MB) in the target file we last (re)opened at
 
-    private final List<Finished> finishedWriters = new ArrayList<>();
-    // as writers are closed from finishedWriters, their last readers are moved into discard, so that abort can cleanup
-    // after us safely; we use a set so we can add in both prepareToCommit and abort
-    private final Set<SSTableReader> discard = new HashSet<>();
-    // true for operations that are performed without Cassandra running (prevents updates of DataTracker)
-    private final boolean isOffline;
+    private final List<SSTableWriter> writers = new ArrayList<>();
+    private final boolean isOffline; // true for operations that are performed without Cassandra running (prevents updates of Tracker)
 
     private SSTableWriter writer;
     private Map<DecoratedKey, RowIndexEntry> cachedKeys = new HashMap<>();
@@ -101,15 +89,11 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
     // for testing (TODO: remove when have byteman setup)
     private boolean throwEarly, throwLate;
 
-    public SSTableRewriter(ColumnFamilyStore cfs, Set<SSTableReader> rewriting, long maxAge, boolean isOffline)
+    public SSTableRewriter(ColumnFamilyStore cfs, LifecycleTransaction transaction, long maxAge, boolean isOffline)
     {
-        this.rewriting = rewriting;
-        for (SSTableReader sstable : rewriting)
-        {
-            originalStarts.put(sstable.descriptor, sstable.first);
+        this.transaction = transaction;
+        for (SSTableReader sstable : this.transaction.originals())
             fileDescriptors.put(sstable.descriptor, CLibrary.getfd(sstable.getFilename()));
-        }
-        this.dataTracker = cfs.getDataTracker();
         this.cfs = cfs;
         this.maxAge = maxAge;
         this.isOffline = isOffline;
@@ -134,7 +118,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
             else
             {
                 boolean save = false;
-                for (SSTableReader reader : rewriting)
+                for (SSTableReader reader : transaction.originals())
                 {
                     if (reader.getCachedPosition(row.key, false) != null)
                     {
@@ -170,7 +154,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
         {
             if (isOffline)
             {
-                for (SSTableReader reader : rewriting)
+                for (SSTableReader reader : transaction.originals())
                 {
                     RowIndexEntry index = reader.getPosition(key, SSTableReader.Operator.GE);
                     CLibrary.trySkipCache(fileDescriptors.get(reader.descriptor), 0, index == null ? 0 : index.position);
@@ -181,10 +165,10 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
                 SSTableReader reader = writer.setMaxDataAge(maxAge).openEarly();
                 if (reader != null)
                 {
-                    replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
-                    currentlyOpenedEarly = reader;
+                    transaction.update(reader, false);
                     currentlyOpenedEarlyAt = writer.getFilePointer();
-                    moveStarts(reader, reader.last, false);
+                    moveStarts(reader, reader.last);
+                    transaction.checkpoint();
                 }
             }
         }
@@ -192,59 +176,19 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
 
     protected Throwable doAbort(Throwable accumulate)
     {
-        try
-        {
-            moveStarts(null, null, true);
-        }
-        catch (Throwable t)
-        {
-            accumulate = merge(accumulate, t);
-        }
-
-        // cleanup any sstables we prepared for commit
-        for (SSTableReader sstable : preparedForCommit)
-        {
-            try
-            {
-                sstable.markObsolete();
-                sstable.selfRef().release();
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate , t);
-            }
-        }
-
-        // abort the writers, and add the early opened readers to our discard pile
-
-        if (writer != null)
-            finishedWriters.add(new Finished(writer, currentlyOpenedEarly));
-
-        for (Finished finished : finishedWriters)
-        {
-            accumulate = finished.writer.abort(accumulate);
-
-            // if we've already been opened, add ourselves to the discard pile
-            if (finished.reader != null)
-                discard.add(finished.reader);
-        }
-
-        accumulate = replaceWithFinishedReaders(Collections.<SSTableReader>emptyList(), accumulate);
+        // abort the writers
+        for (SSTableWriter writer : writers)
+            accumulate = writer.abort(accumulate);
+        // abort the lifecycle transaction
+        accumulate = transaction.abort(accumulate);
         return accumulate;
     }
 
     protected Throwable doCommit(Throwable accumulate)
     {
-        for (Finished f : finishedWriters)
-            accumulate = f.writer.commit(accumulate);
-        accumulate = replaceWithFinishedReaders(preparedForCommit, accumulate);
-
-        return accumulate;
-    }
-
-    protected Throwable doCleanup(Throwable accumulate)
-    {
-        // we have no state of our own to cleanup; Transactional objects cleanup their own state in abort or commit
+        for (SSTableWriter writer : writers)
+            accumulate = writer.commit(accumulate);
+        accumulate = transaction.commit(accumulate);
         return accumulate;
     }
 
@@ -260,100 +204,70 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
      *
      * @param newReader the rewritten reader that replaces them for this region
      * @param lowerbound if !reset, must be non-null, and marks the exclusive lowerbound of the start for each sstable
-     * @param reset true iff we are restoring earlier starts (increasing the range over which they are valid)
      */
-    private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound, boolean reset)
+    private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound)
     {
         if (isOffline)
             return;
         if (preemptiveOpenInterval == Long.MAX_VALUE)
             return;
 
-        List<SSTableReader> toReplace = new ArrayList<>();
-        List<SSTableReader> replaceWith = new ArrayList<>();
         final List<DecoratedKey> invalidateKeys = new ArrayList<>();
-        if (!reset)
-        {
-            invalidateKeys.addAll(cachedKeys.keySet());
-            for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
-                newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
-        }
+        invalidateKeys.addAll(cachedKeys.keySet());
+        for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
+            newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());
 
         cachedKeys = new HashMap<>();
-        for (SSTableReader sstable : ImmutableList.copyOf(rewriting))
+        for (SSTableReader sstable : transaction.originals())
         {
             // we call getCurrentReplacement() to support multiple rewriters operating over the same source readers at once.
             // note: only one such writer should be written to at any moment
-            final SSTableReader latest = sstable.getCurrentReplacement();
-            SSTableReader replacement;
-            if (reset)
-            {
-                DecoratedKey newStart = originalStarts.get(sstable.descriptor);
-                replacement = latest.cloneWithNewStart(newStart, null);
-            }
-            else
-            {
-                // skip any sstables that we know to already be shadowed
-                if (latest.openReason == SSTableReader.OpenReason.SHADOWED)
-                    continue;
-                if (latest.first.compareTo(lowerbound) > 0)
-                    continue;
+            final SSTableReader latest = transaction.current(sstable);
 
-                final Runnable runOnClose = new Runnable()
-                {
-                    public void run()
-                    {
-                        // this is somewhat racey, in that we could theoretically be closing this old reader
-                        // when an even older reader is still in use, but it's not likely to have any major impact
-                        for (DecoratedKey key : invalidateKeys)
-                            latest.invalidateCacheKey(key);
-                    }
-                };
+            // skip any sstables that we know to already be shadowed
+            if (latest.first.compareTo(lowerbound) > 0)
+                continue;
 
-                if (lowerbound.compareTo(latest.last) >= 0)
+            final Runnable runOnClose = new Runnable()
+            {
+                public void run()
                 {
-                    replacement = latest.cloneAsShadowed(runOnClose);
+                    // this is somewhat racey, in that we could theoretically be closing this old reader
+                    // when an even older reader is still in use, but it's not likely to have any major impact
+                    for (DecoratedKey key : invalidateKeys)
+                        latest.invalidateCacheKey(key);
                 }
-                else
+            };
+
+            if (lowerbound.compareTo(latest.last) >= 0)
+            {
+                if (!transaction.isObsolete(latest))
                 {
-                    DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
-                    assert newStart != null;
-                    replacement = latest.cloneWithNewStart(newStart, runOnClose);
+                    latest.runOnClose(runOnClose);
+                    transaction.obsolete(latest);
                 }
+                continue;
             }
 
-            toReplace.add(latest);
-            replaceWith.add(replacement);
-            rewriting.remove(sstable);
-            rewriting.add(replacement);
+            DecoratedKey newStart = latest.firstKeyBeyond(lowerbound);
+            assert newStart != null;
+            SSTableReader replacement = latest.cloneWithNewStart(newStart, runOnClose);
+            transaction.update(replacement, true);
         }
-        cfs.getDataTracker().replaceWithNewInstances(toReplace, replaceWith);
-    }
-
-    private void replaceEarlyOpenedFile(SSTableReader toReplace, SSTableReader replaceWith)
-    {
-        if (isOffline)
-            return;
-        Set<SSTableReader> toReplaceSet;
-        if (toReplace != null)
-        {
-            toReplace.setReplacedBy(replaceWith);
-            toReplaceSet = Collections.singleton(toReplace);
-        }
-        else
-        {
-            dataTracker.markCompacting(Collections.singleton(replaceWith), true, isOffline);
-            toReplaceSet = Collections.emptySet();
-        }
-        dataTracker.replaceEarlyOpenedFiles(toReplaceSet, Collections.singleton(replaceWith));
     }
 
     public void switchWriter(SSTableWriter newWriter)
     {
+        if (newWriter != null)
+            writers.add(newWriter.setMaxDataAge(maxAge));
+
         if (writer == null || writer.getFilePointer() == 0)
         {
             if (writer != null)
+            {
                 writer.abort();
+                writers.remove(writer);
+            }
             writer = newWriter;
             return;
         }
@@ -361,14 +275,13 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
         SSTableReader reader = null;
         if (preemptiveOpenInterval != Long.MAX_VALUE)
         {
-            // we leave it as a tmp file, but we open it and add it to the dataTracker
+            // we leave it as a tmp file, but we open it and add it to the Tracker
             reader = writer.setMaxDataAge(maxAge).openFinalEarly();
-            replaceEarlyOpenedFile(currentlyOpenedEarly, reader);
-            moveStarts(reader, reader.last, false);
+            transaction.update(reader, false);
+            moveStarts(reader, reader.last);
+            transaction.checkpoint();
         }
-        finishedWriters.add(new Finished(writer, reader));
 
-        currentlyOpenedEarly = null;
         currentlyOpenedEarlyAt = 0;
         writer = newWriter;
     }
@@ -387,12 +300,12 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
     /**
      * Finishes the new file(s)
      *
-     * Creates final files, adds the new files to the dataTracker (via replaceReader).
+     * Creates final files, adds the new files to the Tracker (via replaceReader).
      *
      * We add them to the tracker to be able to get rid of the tmpfiles
      *
      * It is up to the caller to do the compacted sstables replacement
-     * gymnastics (ie, call DataTracker#markCompactedSSTablesReplaced(..))
+     * gymnastics (ie, call Tracker#markCompactedSSTablesReplaced(..))
      *
      *
      */
@@ -402,6 +315,7 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
         return finished();
     }
 
+    // returns, in list form, the
     public List<SSTableReader> finished()
     {
         assert state() == State.COMMITTED || state() == State.READY_TO_COMMIT;
@@ -416,82 +330,31 @@ public class SSTableRewriter extends Transactional.AbstractTransactional impleme
             throw new RuntimeException("exception thrown early in finish, for testing");
 
         // No early open to finalize and replace
-        for (Finished f : finishedWriters)
+        for (SSTableWriter writer : writers)
         {
-            if (f.reader != null)
-                discard.add(f.reader);
-
-            f.writer.setRepairedAt(repairedAt).setMaxDataAge(maxAge).setOpenResult(true).prepareToCommit();
-            SSTableReader newReader = f.writer.finished();
-
-            if (f.reader != null)
-                f.reader.setReplacedBy(newReader);
-
-            preparedForCommit.add(newReader);
+            assert writer.getFilePointer() > 0;
+            writer.setRepairedAt(repairedAt).setOpenResult(true).prepareToCommit();
+            SSTableReader reader = writer.finished();
+            transaction.update(reader, false);
+            preparedForCommit.add(reader);
         }
+        transaction.checkpoint();
 
         if (throwLate)
             throw new RuntimeException("exception thrown after all sstables finished, for testing");
-    }
 
-    @VisibleForTesting
-    void throwDuringPrepare(boolean throwEarly)
-    {
-        this.throwEarly = throwEarly;
-        this.throwLate = !throwEarly;
-    }
+        // TODO: do we always want to avoid obsoleting if offline?
+        if (!isOffline)
+            transaction.obsoleteOriginals();
 
-    // cleanup all our temporary readers and swap in our new ones
-    private Throwable replaceWithFinishedReaders(List<SSTableReader> finished, Throwable accumulate)
-    {
-        if (isOffline)
-        {
-            for (SSTableReader reader : discard)
-            {
-                try
-                {
-                    if (reader.getCurrentReplacement() == reader)
-                        reader.markObsolete();
-                }
-                catch (Throwable t)
-                {
-                    accumulate = merge(accumulate, t);
-                }
-            }
-            accumulate = Refs.release(Refs.selfRefs(discard), accumulate);
-        }
-        else
-        {
-            try
-            {
-                dataTracker.replaceEarlyOpenedFiles(discard, finished);
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-            try
-            {
-                dataTracker.unmarkCompacting(discard);
-            }
-            catch (Throwable t)
-            {
-                accumulate = merge(accumulate, t);
-            }
-        }
-        discard.clear();
-        return accumulate;
+        transaction.prepareToCommit();
     }
 
-    private static final class Finished
+    public void throwDuringPrepare(boolean earlyException)
     {
-        final SSTableWriter writer;
-        final SSTableReader reader;
-
-        private Finished(SSTableWriter writer, SSTableReader reader)
-        {
-            this.writer = writer;
-            this.reader = reader;
-        }
+        if (earlyException)
+            throwEarly = true;
+        else
+            throwLate = true;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 23c27b0..8e701b3 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -46,6 +46,7 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.lifecycle.Tracker;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.io.compress.CompressionMetadata;
 import org.apache.cassandra.io.sstable.*;
@@ -122,7 +123,7 @@ import static org.apache.cassandra.db.Directories.SECONDARY_INDEX_NAME_SEPARATOR
  * managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
  * cleaned up safely and can be debugged otherwise.
  *
- * TODO: fill in details about DataTracker and lifecycle interactions for tools, and for compaction strategies
+ * TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategies
  */
 public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
 {
@@ -141,6 +142,9 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         }
     };
 
+    // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
+    public static final class UniqueIdentifier {}
+
     public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
     {
         public int compare(SSTableReader o1, SSTableReader o2)
@@ -170,11 +174,11 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         NORMAL,
         EARLY,
         METADATA_CHANGE,
-        MOVED_START,
-        SHADOWED // => MOVED_START past end
+        MOVED_START
     }
 
     public final OpenReason openReason;
+    public final UniqueIdentifier instanceId = new UniqueIdentifier();
 
     // indexfile and datafile: might be null before a call to load()
     protected SegmentedFile ifile;
@@ -594,9 +598,22 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         return ifile.path();
     }
 
-    public void setTrackedBy(DataTracker tracker)
+    // this is only used for restoring tracker state at delete (and wiring up the keycache) and so
+    // should only be called once it is actually added to the tracker
+    public void setupDeleteNotification(Tracker tracker)
     {
         tidy.type.deletingTask.setTracker(tracker);
+        setupKeyCache();
+    }
+
+    @VisibleForTesting
+    public boolean isDeleteNotificationSetup()
+    {
+        return tidy.type.deletingTask.getTracker() != null;
+    }
+
+    public void setupKeyCache()
+    {
         // under normal operation we can do this at any time, but SSTR is also used outside C* proper,
         // e.g. by BulkLoader, which does not initialize the cache.  As a kludge, we set up the cache
         // here when we know we're being wired into the rest of the server infrastructure.
@@ -908,15 +925,38 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         }
     }
 
-    public void setReplacedBy(SSTableReader replacement)
+    public void setReplaced()
     {
         synchronized (tidy.global)
         {
-            assert replacement != null;
             assert !tidy.isReplaced;
-            assert tidy.global.live == this;
             tidy.isReplaced = true;
-            tidy.global.live = replacement;
+        }
+    }
+
+    public boolean isReplaced()
+    {
+        synchronized (tidy.global)
+        {
+            return tidy.isReplaced;
+        }
+    }
+
+    public void runOnClose(final Runnable runOnClose)
+    {
+        synchronized (tidy.global)
+        {
+            final Runnable existing = tidy.runOnClose;
+            tidy.runOnClose = existing == null
+                              ? runOnClose
+                              : new Runnable()
+                                {
+                                    public void run()
+                                    {
+                                        existing.run();
+                                        runOnClose.run();
+                                    }
+                                };
         }
     }
 
@@ -948,32 +988,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
 
             replacement.first = newStart;
             replacement.last = this.last;
-            setReplacedBy(replacement);
-            return replacement;
-        }
-    }
-
-    public SSTableReader cloneAsShadowed(final Runnable runOnClose)
-    {
-        synchronized (tidy.global)
-        {
-            assert openReason != OpenReason.EARLY;
-            this.tidy.runOnClose = new Runnable()
-            {
-                public void run()
-                {
-                    dfile.dropPageCache(0);
-                    ifile.dropPageCache(0);
-                    runOnClose.run();
-                }
-            };
-
-            SSTableReader replacement = internalOpen(descriptor, components, metadata, partitioner, ifile.sharedCopy(),
-                                                          dfile.sharedCopy(), indexSummary.sharedCopy(), bf.sharedCopy(),
-                                                          maxDataAge, sstableMetadata, OpenReason.SHADOWED);
-            replacement.first = first;
-            replacement.last = last;
-            setReplacedBy(replacement);
             return replacement;
         }
     }
@@ -1036,7 +1050,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                                      sstableMetadata, OpenReason.METADATA_CHANGE);
             replacement.first = this.first;
             replacement.last = this.last;
-            setReplacedBy(replacement);
             return replacement;
         }
     }
@@ -1520,7 +1533,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
      * except for threads holding a reference.
      *
      * @return true if the this is the first time the file was marked obsolete.  Calling this
-     * multiple times is usually buggy (see exceptions in DataTracker.unmarkCompacting and removeOldSSTablesSize).
+     * multiple times is usually buggy (see exceptions in Tracker.unmarkCompacting and removeOldSSTablesSize).
      */
     public boolean markObsolete()
     {
@@ -1638,11 +1651,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         return sstableMetadata.repairedAt != ActiveRepairService.UNREPAIRED_SSTABLE;
     }
 
-    public SSTableReader getCurrentReplacement()
-    {
-        return tidy.global.live;
-    }
-
     /**
      * TODO: Move someplace reusable
      */
@@ -2048,8 +2056,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         static final ConcurrentMap<Descriptor, Ref<GlobalTidy>> lookup = new ConcurrentHashMap<>();
 
         private final Descriptor desc;
-        // a single convenience property for getting the most recent version of an sstable, not related to tidying
-        private SSTableReader live;
         // the readMeter that is shared between all instances of the sstable, and can be overridden in all of them
         // at once also, for testing purposes
         private RestorableMeter readMeter;
@@ -2064,7 +2070,6 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         {
             this.desc = reader.descriptor;
             this.isCompacted = new AtomicBoolean();
-            this.live = reader;
         }
 
         void ensureReadMeter()
@@ -2128,6 +2133,13 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
         }
     }
 
+    @VisibleForTesting
+    public static void resetTidying()
+    {
+        GlobalTidy.lookup.clear();
+        DescriptorTypeTidy.lookup.clear();
+    }
+
     public static abstract class Factory
     {
         public abstract SSTableReader open(final Descriptor descriptor,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index fa17c20..a7a7fcc 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -377,7 +377,8 @@ public class BigTableWriter extends SSTableWriter
             return accumulate;
         }
 
-        protected Throwable doCleanup(Throwable accumulate)
+        @Override
+        protected Throwable doPreCleanup(Throwable accumulate)
         {
             accumulate = dbuilder.close(accumulate);
             return accumulate;
@@ -562,7 +563,8 @@ public class BigTableWriter extends SSTableWriter
             return indexFile.abort(accumulate);
         }
 
-        protected Throwable doCleanup(Throwable accumulate)
+        @Override
+        protected Throwable doPreCleanup(Throwable accumulate)
         {
             accumulate = summary.close(accumulate);
             accumulate = bf.close(accumulate);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index d63be31..3c35a34 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -77,7 +77,7 @@ public class SequentialWriter extends OutputStream implements WritableByteChanne
     protected class TransactionalProxy extends AbstractTransactional
     {
         @Override
-        protected Throwable doCleanup(Throwable accumulate)
+        protected Throwable doPreCleanup(Throwable accumulate)
         {
             if (directoryFD >= 0)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index 842d06d..4ab4446 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -215,28 +215,28 @@ public class ColumnFamilyMetrics
         {
             public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
+                return cfs.getTracker().getView().getCurrentMemtable().getOperations();
             }
         });
         memtableOnHeapSize = createColumnFamilyGauge("MemtableOnHeapSize", new Gauge<Long>()
         {
             public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
             }
         });
         memtableOffHeapSize = createColumnFamilyGauge("MemtableOffHeapSize", new Gauge<Long>()
         {
             public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+                return cfs.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
             }
         });
         memtableLiveDataSize = createColumnFamilyGauge("MemtableLiveDataSize", new Gauge<Long>()
         {
             public Long getValue()
             {
-                return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+                return cfs.getTracker().getView().getCurrentMemtable().getLiveDataSize();
             }
         });
         allMemtablesOnHeapSize = createColumnFamilyGauge("AllMemtablesHeapSize", new Gauge<Long>()
@@ -245,7 +245,7 @@ public class ColumnFamilyMetrics
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().onHeap().owns();
                 return size;
             }
         });
@@ -255,7 +255,7 @@ public class ColumnFamilyMetrics
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getAllocator().offHeap().owns();
                 return size;
             }
         });
@@ -265,7 +265,7 @@ public class ColumnFamilyMetrics
             {
                 long size = 0;
                 for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
-                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+                    size += cfs2.getTracker().getView().getCurrentMemtable().getLiveDataSize();
                 return size;
             }
         });
@@ -288,7 +288,7 @@ public class ColumnFamilyMetrics
             public Long getValue()
             {
                 long memtablePartitions = 0;
-                for (Memtable memtable : cfs.getDataTracker().getView().getAllMemtables())
+                for (Memtable memtable : cfs.getTracker().getView().getAllMemtables())
                     memtablePartitions += memtable.partitionCount();
                 return SSTableReader.getApproximateKeyCount(cfs.getSSTables()) + memtablePartitions;
             }
@@ -358,7 +358,7 @@ public class ColumnFamilyMetrics
         {
             public Integer getValue()
             {
-                return cfs.getDataTracker().getSSTables().size();
+                return cfs.getTracker().getSSTables().size();
             }
         });
         liveDiskSpaceUsed = createColumnFamilyCounter("LiveDiskSpaceUsed");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
index 6a70692..44522db 100644
--- a/src/java/org/apache/cassandra/streaming/StreamSession.java
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -28,13 +28,14 @@ import javax.annotation.Nullable;
 
 import com.google.common.base.Function;
 import com.google.common.collect.*;
+
+import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DataTracker;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.dht.AbstractBounds;
@@ -318,9 +319,9 @@ public class StreamSession implements IEndpointStateChangeSubscriber
                 final List<AbstractBounds<RowPosition>> rowBoundsList = new ArrayList<>(ranges.size());
                 for (Range<Token> range : ranges)
                     rowBoundsList.add(Range.makeRowRange(range));
-                refs.addAll(cfStore.selectAndReference(new Function<DataTracker.View, List<SSTableReader>>()
+                refs.addAll(cfStore.selectAndReference(new Function<View, List<SSTableReader>>()
                 {
-                    public List<SSTableReader> apply(DataTracker.View view)
+                    public List<SSTableReader> apply(View view)
                     {
                         List<SSTableReader> filteredSSTables = ColumnFamilyStore.CANONICAL_SSTABLES.apply(view);
                         Set<SSTableReader> sstables = Sets.newHashSet();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 9f26637..d32ef88 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -32,12 +32,8 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
-import org.apache.cassandra.db.compaction.LeveledManifest;
-import org.apache.cassandra.db.compaction.Scrubber;
-import org.apache.cassandra.db.compaction.WrappingCompactionStrategy;
+import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
@@ -121,9 +117,9 @@ public class StandaloneScrubber
             {
                 for (SSTableReader sstable : sstables)
                 {
-                    try
+                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, true, !options.noValidate);
                         try
                         {
                             scrubber.scrub();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
index 2541d6e..e881133 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java
@@ -27,10 +27,12 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.commons.cli.*;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SSTableSplitter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
@@ -145,12 +147,11 @@ public class StandaloneSplitter
             if (options.snapshot)
                 System.out.println(String.format("Pre-split sstables snapshotted into snapshot %s", snapshotName));
 
-            cfs.getDataTracker().markCompacting(sstables, false, true);
             for (SSTableReader sstable : sstables)
             {
-                try
+                try (LifecycleTransaction transaction = LifecycleTransaction.offline(OperationType.UNKNOWN, sstable))
                 {
-                    new SSTableSplitter(cfs, sstable, options.sizeInMB).split();
+                    new SSTableSplitter(cfs, transaction, options.sizeInMB).split();
 
                     // Remove the sstable (it's been copied by split and snapshotted)
                     sstable.markObsolete();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
index 409a5f0..626d429 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java
@@ -29,7 +29,9 @@ import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.Upgrader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
@@ -98,9 +100,9 @@ public class StandaloneUpgrader
 
             for (SSTableReader sstable : readers)
             {
-                try
+                try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.UPGRADE_SSTABLES, sstable))
                 {
-                    Upgrader upgrader = new Upgrader(cfs, sstable, handler);
+                    Upgrader upgrader = new Upgrader(cfs, txn, handler);
                     upgrader.upgrade();
 
                     if (!options.keepSource)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Blocker.java b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
new file mode 100644
index 0000000..5192e98
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/Blocker.java
@@ -0,0 +1,63 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra.utils.concurrent;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class Blocker
+{
+    private final ReentrantLock lock = new ReentrantLock();
+    private final Condition unblocked = lock.newCondition();
+    private volatile boolean block = false;
+
+    public void block(boolean block)
+    {
+        this.block = block;
+        if (!block)
+        {
+            lock.lock();
+            try
+            {
+                unblocked.signalAll();
+            }
+            finally
+            {
+                lock.unlock();
+            }
+        }
+    }
+
+    public void ask()
+    {
+        if (block)
+        {
+            lock.lock();
+            try
+            {
+                while (block)
+                    unblocked.awaitUninterruptibly();
+            }
+            finally
+            {
+                lock.unlock();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
index bcf5095..5b0eb8e 100644
--- a/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
+++ b/src/java/org/apache/cassandra/utils/concurrent/Transactional.java
@@ -70,6 +70,7 @@ public interface Transactional extends AutoCloseable
             ABORTED;
         }
 
+        private boolean permitRedundantTransitions;
         private State state = State.IN_PROGRESS;
 
         // the methods for actually performing the necessary behaviours, that are themselves protected against
@@ -79,9 +80,18 @@ public interface Transactional extends AutoCloseable
         protected abstract Throwable doCommit(Throwable accumulate);
         protected abstract Throwable doAbort(Throwable accumulate);
 
-        // this only needs to perform cleanup of state unique to this instance; any internal
+        // these only needs to perform cleanup of state unique to this instance; any internal
         // Transactional objects will perform cleanup in the commit() or abort() calls
-        protected abstract Throwable doCleanup(Throwable accumulate);
+
+        /**
+         * perform an exception-safe pre-abort cleanup; this will still be run *after* commit
+         */
+        protected Throwable doPreCleanup(Throwable accumulate){ return accumulate; }
+
+        /**
+         * perform an exception-safe post-abort cleanup
+         */
+        protected Throwable doPostCleanup(Throwable accumulate){ return accumulate; }
 
         /**
          * Do any preparatory work prior to commit. This method should throw any exceptions that can be encountered
@@ -94,10 +104,13 @@ public interface Transactional extends AutoCloseable
          */
         public final Throwable commit(Throwable accumulate)
         {
+            if (permitRedundantTransitions && state == State.COMMITTED)
+                return accumulate;
             if (state != State.READY_TO_COMMIT)
-                throw new IllegalStateException("Commit attempted before prepared to commit");
+                throw new IllegalStateException("Cannot commit unless READY_TO_COMMIT; state is " + state);
             accumulate = doCommit(accumulate);
-            accumulate = doCleanup(accumulate);
+            accumulate = doPreCleanup(accumulate);
+            accumulate = doPostCleanup(accumulate);
             state = State.COMMITTED;
             return accumulate;
         }
@@ -123,8 +136,9 @@ public interface Transactional extends AutoCloseable
             }
             state = State.ABORTED;
             // we cleanup first so that, e.g., file handles can be released prior to deletion
-            accumulate = doCleanup(accumulate);
+            accumulate = doPreCleanup(accumulate);
             accumulate = doAbort(accumulate);
+            accumulate = doPostCleanup(accumulate);
             return accumulate;
         }
 
@@ -147,6 +161,8 @@ public interface Transactional extends AutoCloseable
          */
         public final void prepareToCommit()
         {
+            if (permitRedundantTransitions && state == State.READY_TO_COMMIT)
+                return;
             if (state != State.IN_PROGRESS)
                 throw new IllegalStateException("Cannot prepare to commit unless IN_PROGRESS; state is " + state);
 
@@ -183,6 +199,11 @@ public interface Transactional extends AutoCloseable
         {
             return state;
         }
+
+        protected void permitRedundantTransitions()
+        {
+            permitRedundantTransitions = true;
+        }
     }
 
     // commit should generally never throw an exception, and preferably never generate one,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index bf71639..e6c8f56 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -18,9 +18,7 @@
 */
 package org.apache.cassandra.db.compaction;
 
-import java.io.IOException;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
@@ -35,6 +33,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.sstable.SSTableUtils;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -126,8 +125,11 @@ public class LongCompactionsTest
 
         long start = System.nanoTime();
         final int gcBefore = (int) (System.currentTimeMillis() / 1000) - Schema.instance.getCFMetaData(KEYSPACE1, "Standard1").getGcGraceSeconds();
-        assert store.getDataTracker().markCompacting(sstables): "Cannot markCompacting all sstables";
-        new CompactionTask(store, sstables, gcBefore, false).execute(null);
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.COMPACTION))
+        {
+            assert txn != null : "Cannot markCompacting all sstables";
+            new CompactionTask(store, txn, gcBefore, false).execute(null);
+        }
         System.out.println(String.format("%s: sstables=%d rowsper=%d colsper=%d: %d ms",
                                          this.getClass().getName(),
                                          sstableCount,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/MockSchema.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/MockSchema.java b/test/unit/org/apache/cassandra/MockSchema.java
new file mode 100644
index 0000000..bc236e1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/MockSchema.java
@@ -0,0 +1,167 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
+package org.apache.cassandra;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.SimpleSparseCellNameType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.BufferedSegmentedFile;
+import org.apache.cassandra.io.util.ChannelProxy;
+import org.apache.cassandra.io.util.Memory;
+import org.apache.cassandra.io.util.SegmentedFile;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class MockSchema
+{
+    static
+    {
+        Memory offsets = Memory.allocate(4);
+        offsets.setInt(0, 0);
+        indexSummary = new IndexSummary(Murmur3Partitioner.instance, offsets, 0, Memory.allocate(4), 0, 0, 0, 1);
+    }
+    private static final AtomicInteger id = new AtomicInteger();
+    public static final Keyspace ks = Keyspace.mockKS(new KSMetaData("mockks", SimpleStrategy.class, ImmutableMap.of("replication_factor", "1"), false));
+    public static final ColumnFamilyStore cfs = newCFS();
+
+    private static final IndexSummary indexSummary;
+    private static final SegmentedFile segmentedFile = new BufferedSegmentedFile(new ChannelProxy(temp("mocksegmentedfile")), 0);
+
+    public static Memtable memtable()
+    {
+        return new Memtable(cfs.metadata);
+    }
+
+    public static SSTableReader sstable(int generation)
+    {
+        return sstable(generation, false);
+    }
+
+    public static SSTableReader sstable(int generation, boolean keepRef)
+    {
+        return sstable(generation, 0, keepRef);
+    }
+
+    public static SSTableReader sstable(int generation, int size)
+    {
+        return sstable(generation, size, false);
+    }
+
+    public static SSTableReader sstable(int generation, int size, boolean keepRef)
+    {
+        return sstable(generation, size, keepRef, cfs);
+    }
+    public static SSTableReader sstable(int generation, int size, boolean keepRef, ColumnFamilyStore cfs)
+    {
+        Descriptor descriptor = new Descriptor(temp("mockcfdir").getParentFile(), "mockks", "mockcf", generation, Descriptor.Type.FINAL);
+        Set<Component> components = ImmutableSet.of(Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.TOC);
+        for (Component component : components)
+        {
+            File file = new File(descriptor.filenameFor(component));
+            try
+            {
+                file.createNewFile();
+            }
+            catch (IOException e)
+            {
+            }
+            file.deleteOnExit();
+        }
+        if (size > 0)
+        {
+            try
+            {
+                File file = new File(descriptor.filenameFor(Component.DATA));
+                try (RandomAccessFile raf = new RandomAccessFile(file, "rw"))
+                {
+                    raf.setLength(size);
+                }
+            }
+            catch (IOException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+        StatsMetadata metadata = (StatsMetadata) new MetadataCollector(cfs.metadata.comparator)
+                                                 .finalizeMetadata(Murmur3Partitioner.instance.getClass().getCanonicalName(), 0.01f, -1)
+                                                 .get(MetadataType.STATS);
+        SSTableReader reader = SSTableReader.internalOpen(descriptor, components, cfs.metadata, Murmur3Partitioner.instance,
+                                                          segmentedFile.sharedCopy(), segmentedFile.sharedCopy(), indexSummary.sharedCopy(),
+                                                          new AlwaysPresentFilter(), 1L, metadata, SSTableReader.OpenReason.NORMAL);
+        reader.first = reader.last = readerBounds(generation);
+        if (!keepRef)
+            reader.selfRef().release();
+        return reader;
+    }
+
+    public static ColumnFamilyStore newCFS()
+    {
+        String cfname = "mockcf" + (id.incrementAndGet());
+        CFMetaData metadata = newCFMetaData(cfname);
+        return new ColumnFamilyStore(ks, cfname, Murmur3Partitioner.instance, 0, metadata, new Directories(metadata), false, false);
+    }
+
+    private static CFMetaData newCFMetaData(String cfname)
+    {
+        CFMetaData metadata = new CFMetaData("mockks", cfname, ColumnFamilyType.Standard, new SimpleSparseCellNameType(UTF8Type.instance));
+        metadata.caching(CachingOptions.NONE);
+        return metadata;
+    }
+
+    public static BufferDecoratedKey readerBounds(int generation)
+    {
+        return new BufferDecoratedKey(new Murmur3Partitioner.LongToken(generation), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    private static File temp(String id)
+    {
+        try
+        {
+            File file = File.createTempFile(id, "tmp");
+            file.deleteOnExit();
+            return file;
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 9b8e5df..c2205c4 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -24,16 +24,17 @@ import java.io.*;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.UUID;
+import java.nio.channels.FileChannel;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
@@ -44,17 +45,27 @@ import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
+import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.IndexSummary;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.sstable.format.big.BigTableReader;
+import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
+import org.apache.cassandra.io.sstable.metadata.MetadataType;
+import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.io.util.*;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.AlwaysPresentFilter;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
+import org.apache.hadoop.fs.FileUtil;
 
 import static org.junit.Assert.assertTrue;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index e5fd470..27e7a2b 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -43,7 +43,6 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
 
 import org.apache.cassandra.utils.concurrent.Refs;
 import static org.junit.Assert.assertEquals;
@@ -171,7 +170,7 @@ public class KeyCacheTest
 
         assertKeyCacheSize(2, KEYSPACE1, COLUMN_FAMILY1);
 
-        Set<SSTableReader> readers = cfs.getDataTracker().getSSTables();
+        Set<SSTableReader> readers = cfs.getTracker().getSSTables();
         Refs<SSTableReader> refs = Refs.tryRef(readers);
         if (refs == null)
             throw new IllegalStateException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index a5af823..dbbce9e 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -20,57 +20,63 @@ package org.apache.cassandra.db;
  *
  */
 
-import java.io.*;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.ExecutionException;
 
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.format.SSTableFormat;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.utils.UUIDGen;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.Scrubber;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableRewriter;
-import org.apache.cassandra.io.sstable.format.SSTableFormat;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.UUIDGen;
 
 import static org.junit.Assert.*;
 import static org.junit.Assume.assumeTrue;
 
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class ScrubTest
@@ -155,7 +161,8 @@ public class ScrubTest
         overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
 
         // with skipCorrupted == false, the scrub is expected to fail
-        try(Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true))
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, false, true);)
         {
             scrubber.scrub();
             fail("Expected a CorruptSSTableException to be thrown");
@@ -164,7 +171,8 @@ public class ScrubTest
 
         // with skipCorrupted == true, the corrupt rows will be skipped
         Scrubber.ScrubResult scrubResult;
-        try(Scrubber scrubber = new Scrubber(cfs, sstable, true, false, true))
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, true, false, true);)
         {
             scrubResult = scrubber.scrubWithResult();
         }
@@ -213,20 +221,24 @@ public class ScrubTest
         overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
 
         // with skipCorrupted == false, the scrub is expected to fail
-        Scrubber scrubber = new Scrubber(cfs, sstable, false, false, true);
-        try
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, false, true))
         {
+            // with skipCorrupted == true, the corrupt row will be skipped
             scrubber.scrub();
             fail("Expected a CorruptSSTableException to be thrown");
         }
         catch (IOError err) {}
 
-        // with skipCorrupted == true, the corrupt row will be skipped
-        scrubber = new Scrubber(cfs, sstable, true, false, true);
-        scrubber.scrub();
-        scrubber.close();
-        assertEquals(1, cfs.getSSTables().size());
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(Arrays.asList(sstable), OperationType.SCRUB);
+             Scrubber scrubber = new Scrubber(cfs, txn, true, false, true))
+        {
+            // with skipCorrupted == true, the corrupt row will be skipped
+            scrubber.scrub();
+            scrubber.close();
+        }
 
+        assertEquals(1, cfs.getSSTables().size());
         // verify that we can read all of the rows, and there is now one less row
         rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
         assertEquals(1, rows.size());
@@ -369,9 +381,13 @@ public class ScrubTest
         components.add(Component.STATS);
         components.add(Component.SUMMARY);
         components.add(Component.TOC);
+
         SSTableReader sstable = SSTableReader.openNoValidation(desc, components, cfs);
+        if (sstable.last.compareTo(sstable.first) < 0)
+            sstable.last = sstable.first;
 
-        try(Scrubber scrubber = new Scrubber(cfs, sstable, false, true, true))
+        try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable);
+             Scrubber scrubber = new Scrubber(cfs, txn, false, true, true);)
         {
             scrubber.scrub();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index 1dc72ae..235462b 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import junit.framework.Assert;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.utils.concurrent.Refs;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
-import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -91,13 +90,18 @@ public class AntiCompactionTest
         Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        Refs<SSTableReader> refs = Refs.ref(sstables);
-        long repairedAt = 1000;
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
-
-        assertEquals(2, store.getSSTables().size());
         int repairedKeys = 0;
         int nonRepairedKeys = 0;
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            if (txn == null)
+                throw new IllegalStateException();
+            long repairedAt = 1000;
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+        }
+
+        assertEquals(2, store.getSSTables().size());
         for (SSTableReader sstable : store.getSSTables())
         {
             try (ISSTableScanner scanner = sstable.getScanner())
@@ -123,7 +127,7 @@ public class AntiCompactionTest
             assertFalse(sstable.isMarkedCompacted());
             assertEquals(1, sstable.selfRef().globalCount());
         }
-        assertEquals(0, store.getDataTracker().getCompacting().size());
+        assertEquals(0, store.getTracker().getCompacting().size());
         assertEquals(repairedKeys, 4);
         assertEquals(nonRepairedKeys, 6);
     }
@@ -139,13 +143,16 @@ public class AntiCompactionTest
         long origSize = s.bytesOnDisk();
         Range<Token> range = new Range<Token>(new BytesToken(ByteBufferUtil.bytes(0)), new BytesToken(ByteBufferUtil.bytes(500)));
         Collection<SSTableReader> sstables = cfs.getSSTables();
-        CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), Refs.tryRef(sstables), 12345);
+        try (LifecycleTransaction txn = cfs.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(cfs, Arrays.asList(range), refs, txn, 12345);
+        }
         long sum = 0;
         for (SSTableReader x : cfs.getSSTables())
             sum += x.bytesOnDisk();
         assertEquals(sum, cfs.metric.liveDiskSpaceUsed.getCount());
         assertEquals(origSize, cfs.metric.liveDiskSpaceUsed.getCount(), 100000);
-
     }
 
     private SSTableReader writeFile(ColumnFamilyStore cfs, int count)
@@ -210,10 +217,12 @@ public class AntiCompactionTest
         Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("4".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        Refs<SSTableReader> refs = Refs.tryRef(sstables);
-        Assert.assertNotNull(refs);
         long repairedAt = 1000;
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, repairedAt);
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, repairedAt);
+        }
         /*
         Anticompaction will be anti-compacting 10 SSTables but will be doing this two at a time
         so there will be no net change in the number of sstables
@@ -256,12 +265,16 @@ public class AntiCompactionTest
         Range<Token> range = new Range<Token>(new BytesToken("0".getBytes()), new BytesToken("9999".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        CompactionManager.instance.performAnticompaction(store, ranges, Refs.tryRef(sstables), 1);
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 1);
+        }
 
         assertThat(store.getSSTables().size(), is(1));
         assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(true));
         assertThat(Iterables.get(store.getSSTables(), 0).selfRef().globalCount(), is(1));
-        assertThat(store.getDataTracker().getCompacting().size(), is(0));
+        assertThat(store.getTracker().getCompacting().size(), is(0));
     }
 
 
@@ -282,8 +295,12 @@ public class AntiCompactionTest
         Range<Token> range = new Range<Token>(new BytesToken("-10".getBytes()), new BytesToken("-1".getBytes()));
         List<Range<Token>> ranges = Arrays.asList(range);
 
-        Refs<SSTableReader> refs = Refs.ref(sstables);
-        CompactionManager.instance.performAnticompaction(store, ranges, refs, 0);
+
+        try (LifecycleTransaction txn = store.getTracker().tryModify(sstables, OperationType.ANTICOMPACTION);
+             Refs<SSTableReader> refs = Refs.ref(sstables))
+        {
+            CompactionManager.instance.performAnticompaction(store, ranges, refs, txn, 0);
+        }
 
         assertThat(store.getSSTables().size(), is(10));
         assertThat(Iterables.get(store.getSSTables(), 0).isRepaired(), is(false));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
index 88074af..235fd49 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db.compaction;
 
 import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 
 import com.google.common.primitives.Longs;
 import org.junit.Before;
@@ -41,11 +40,12 @@ import org.apache.cassandra.db.compaction.writers.MajorLeveledCompactionWriter;
 import org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter;
 import org.apache.cassandra.db.compaction.writers.SplittingSizeTieredCompactionWriter;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.utils.ByteBufferUtil;
+
 import static org.junit.Assert.assertEquals;
 
 public class CompactionAwareWriterTest
@@ -81,10 +81,10 @@ public class CompactionAwareWriterTest
         int rowCount = 1000;
         cfs.disableAutoCompaction();
         populate(cfs, rowCount);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
-        long beforeSize = sstables.iterator().next().onDiskLength();
-        CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, sstables, sstables, false, OperationType.COMPACTION);
-        int rows = compact(cfs, sstables, writer);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        CompactionAwareWriter writer = new DefaultCompactionWriter(cfs, txn, txn.originals(), false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
         assertEquals(1, cfs.getSSTables().size());
         assertEquals(rowCount, rows);
         assertEquals(beforeSize, cfs.getSSTables().iterator().next().onDiskLength());
@@ -100,11 +100,11 @@ public class CompactionAwareWriterTest
         cfs.disableAutoCompaction();
         int rowCount = 1000;
         populate(cfs, rowCount);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
-        long beforeSize = sstables.iterator().next().onDiskLength();
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
         int sstableSize = (int)beforeSize/10;
-        CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, sstables, sstables, sstableSize, 0, false, OperationType.COMPACTION);
-        int rows = compact(cfs, sstables, writer);
+        CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, txn, txn.originals(), sstableSize, 0, false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
         assertEquals(10, cfs.getSSTables().size());
         assertEquals(rowCount, rows);
         validateData(cfs, rowCount);
@@ -118,10 +118,10 @@ public class CompactionAwareWriterTest
         cfs.disableAutoCompaction();
         int rowCount = 10000;
         populate(cfs, rowCount);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
-        long beforeSize = sstables.iterator().next().onDiskLength();
-        CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, sstables, sstables, OperationType.COMPACTION, 0);
-        int rows = compact(cfs, sstables, writer);
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
+        CompactionAwareWriter writer = new SplittingSizeTieredCompactionWriter(cfs, txn, txn.originals(), OperationType.COMPACTION, 0);
+        int rows = compact(cfs, txn, writer);
         long expectedSize = beforeSize / 2;
         List<SSTableReader> sortedSSTables = new ArrayList<>(cfs.getSSTables());
 
@@ -154,11 +154,11 @@ public class CompactionAwareWriterTest
         int rowCount = 20000;
         int targetSSTableCount = 50;
         populate(cfs, rowCount);
-        Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
-        long beforeSize = sstables.iterator().next().onDiskLength();
+        LifecycleTransaction txn = cfs.getTracker().tryModify(cfs.getSSTables(), OperationType.COMPACTION);
+        long beforeSize = txn.originals().iterator().next().onDiskLength();
         int sstableSize = (int)beforeSize/targetSSTableCount;
-        CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, sstables, sstables, sstableSize, false, OperationType.COMPACTION);
-        int rows = compact(cfs, sstables, writer);
+        CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, txn, txn.originals(), sstableSize, false, OperationType.COMPACTION);
+        int rows = compact(cfs, txn, writer);
         assertEquals(targetSSTableCount, cfs.getSSTables().size());
         int [] levelCounts = new int[5];
         assertEquals(rowCount, rows);
@@ -175,13 +175,13 @@ public class CompactionAwareWriterTest
         cfs.truncateBlocking();
     }
 
-    private int compact(ColumnFamilyStore cfs, Set<SSTableReader> sstables, CompactionAwareWriter writer)
+    private int compact(ColumnFamilyStore cfs, LifecycleTransaction txn, CompactionAwareWriter writer)
     {
-        assert sstables.size() == 1;
+        assert txn.originals().size() == 1;
         int rowsWritten = 0;
-        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(sstables))
+        try (AbstractCompactionStrategy.ScannerList scanners = cfs.getCompactionStrategy().getScanners(txn.originals()))
         {
-            CompactionController controller = new CompactionController(cfs, sstables, cfs.gcBefore(System.currentTimeMillis()));
+            CompactionController controller = new CompactionController(cfs, txn.originals(), cfs.gcBefore(System.currentTimeMillis()));
             ISSTableScanner scanner = scanners.scanners.get(0);
             while(scanner.hasNext())
             {
@@ -191,7 +191,6 @@ public class CompactionAwareWriterTest
             }
         }
         Collection<SSTableReader> newSSTables = writer.finish();
-        cfs.getDataTracker().markCompactedSSTablesReplaced(sstables, newSSTables, OperationType.COMPACTION);
         return rowsWritten;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
index f1d016b..64e4465 100644
--- a/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/DateTieredCompactionStrategyTest.java
@@ -324,10 +324,10 @@ public class DateTieredCompactionStrategyTest extends SchemaLoader
         Thread.sleep(2000);
         AbstractCompactionTask t = dtcs.getNextBackgroundTask((int) (System.currentTimeMillis()/1000));
         assertNotNull(t);
-        assertEquals(1, Iterables.size(t.sstables));
-        SSTableReader sstable = t.sstables.iterator().next();
+        assertEquals(1, Iterables.size(t.transaction.originals()));
+        SSTableReader sstable = t.transaction.originals().iterator().next();
         assertEquals(sstable, expiredSSTable);
-        cfs.getDataTracker().unmarkCompacting(cfs.getSSTables());
+        t.transaction.abort();
     }
 
 }


[6/7] cassandra git commit: Extend Transactional API to sstable lifecycle management

Posted by be...@apache.org.
Extend Transactional API to sstable lifecycle management

patch by benedict; reviewed by marcus for CASSANDRA-8568


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e5a76bdb
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e5a76bdb
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e5a76bdb

Branch: refs/heads/trunk
Commit: e5a76bdb5fc04ffa16b8becaa7877186226c3b32
Parents: 33d71b8
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Thu Mar 12 10:23:35 2015 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Fri May 22 09:44:36 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  | 249 ++++--
 .../org/apache/cassandra/db/DataTracker.java    | 793 -------------------
 .../cassandra/db/HintedHandOffManager.java      |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |  15 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  22 +-
 .../compaction/AbstractCompactionStrategy.java  |   7 +-
 .../db/compaction/AbstractCompactionTask.java   |  19 +-
 .../db/compaction/CompactionController.java     |   4 +-
 .../db/compaction/CompactionManager.java        | 182 +++--
 .../cassandra/db/compaction/CompactionTask.java |  54 +-
 .../DateTieredCompactionStrategy.java           |  17 +-
 .../compaction/LeveledCompactionStrategy.java   |  23 +-
 .../db/compaction/LeveledCompactionTask.java    |  11 +-
 .../db/compaction/LeveledManifest.java          |  11 +-
 .../db/compaction/SSTableSplitter.java          |  13 +-
 .../cassandra/db/compaction/Scrubber.java       |  21 +-
 .../SizeTieredCompactionStrategy.java           |  30 +-
 .../cassandra/db/compaction/Upgrader.java       |  15 +-
 .../compaction/WrappingCompactionStrategy.java  |   2 +-
 .../writers/CompactionAwareWriter.java          |  11 +-
 .../writers/DefaultCompactionWriter.java        |  11 +-
 .../writers/MajorLeveledCompactionWriter.java   |  11 +-
 .../writers/MaxSSTableSizeWriter.java           |  10 +-
 .../SplittingSizeTieredCompactionWriter.java    |  14 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |   4 +-
 .../db/index/SecondaryIndexManager.java         |   2 +-
 .../apache/cassandra/db/lifecycle/Helpers.java  | 241 ++++++
 .../db/lifecycle/LifecycleTransaction.java      | 511 ++++++++++++
 .../db/lifecycle/SSTableIntervalTree.java       |  40 +
 .../apache/cassandra/db/lifecycle/Tracker.java  | 468 +++++++++++
 .../org/apache/cassandra/db/lifecycle/View.java | 252 ++++++
 .../io/compress/CompressionMetadata.java        |   2 +-
 .../io/sstable/IndexSummaryManager.java         | 106 ++-
 .../io/sstable/SSTableDeletingTask.java         |  27 +-
 .../cassandra/io/sstable/SSTableRewriter.java   | 295 ++-----
 .../io/sstable/format/SSTableReader.java        | 100 ++-
 .../io/sstable/format/big/BigTableWriter.java   |   6 +-
 .../cassandra/io/util/SequentialWriter.java     |   2 +-
 .../cassandra/metrics/ColumnFamilyMetrics.java  |  18 +-
 .../cassandra/streaming/StreamSession.java      |   7 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../cassandra/tools/StandaloneSplitter.java     |   7 +-
 .../cassandra/tools/StandaloneUpgrader.java     |   6 +-
 .../cassandra/utils/concurrent/Blocker.java     |  63 ++
 .../utils/concurrent/Transactional.java         |  31 +-
 .../db/compaction/LongCompactionsTest.java      |  10 +-
 test/unit/org/apache/cassandra/MockSchema.java  | 167 ++++
 test/unit/org/apache/cassandra/Util.java        |  27 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |   3 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |  58 +-
 .../db/compaction/AntiCompactionTest.java       |  51 +-
 .../compaction/CompactionAwareWriterTest.java   |  45 +-
 .../DateTieredCompactionStrategyTest.java       |   6 +-
 .../cassandra/db/lifecycle/HelpersTest.java     | 158 ++++
 .../db/lifecycle/LifecycleTransactionTest.java  | 412 ++++++++++
 .../cassandra/db/lifecycle/TrackerTest.java     | 342 ++++++++
 .../apache/cassandra/db/lifecycle/ViewTest.java | 202 +++++
 .../io/sstable/IndexSummaryManagerTest.java     | 123 ++-
 .../cassandra/io/sstable/SSTableReaderTest.java |  11 +-
 .../io/sstable/SSTableRewriterTest.java         | 250 +++---
 61 files changed, 3902 insertions(+), 1711 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8b59309..ca87385 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2
+ * Extend Transactional API to sstable lifecycle management (CASSANDRA-8568)
  * (cqlsh) Add support for native protocol 4 (CASSANDRA-9399)
  * Ensure that UDF and UDAs are keyspace-isolated (CASSANDRA-9409)
  * Revert CASSANDRA-7807 (tracing completion client notifications) (CASSANDRA-9429)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 4452db2..cc9b26a 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -34,6 +34,10 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.*;
 
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.View;
+import org.apache.cassandra.db.lifecycle.Tracker;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.io.FSWriteError;
 import org.json.simple.*;
 import org.slf4j.Logger;
@@ -80,6 +84,8 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import com.clearspring.analytics.stream.Counter;
 
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+
 public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 {
     private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
@@ -149,12 +155,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     /**
      * Memtables and SSTables on disk for this column family.
      *
-     * We synchronize on the DataTracker to ensure isolation when we want to make sure
+     * We synchronize on the Tracker to ensure isolation when we want to make sure
      * that the memtable we're acting on doesn't change out from under us.  I.e., flush
      * syncronizes on it to make sure it can submit on both executors atomically,
      * so anyone else who wants to make sure flush doesn't interfere should as well.
      */
-    private final DataTracker data;
+    private final Tracker data;
 
     /* The read order, used to track accesses to off-heap memtable storage */
     public final OpOrder readOrdering = new OpOrder();
@@ -288,13 +294,27 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    private ColumnFamilyStore(Keyspace keyspace,
+    public ColumnFamilyStore(Keyspace keyspace,
+                             String columnFamilyName,
+                             IPartitioner partitioner,
+                             int generation,
+                             CFMetaData metadata,
+                             Directories directories,
+                             boolean loadSSTables)
+    {
+        this(keyspace, columnFamilyName, partitioner, generation, metadata, directories, loadSSTables, true);
+    }
+
+
+    @VisibleForTesting
+    public ColumnFamilyStore(Keyspace keyspace,
                               String columnFamilyName,
                               IPartitioner partitioner,
                               int generation,
                               CFMetaData metadata,
                               Directories directories,
-                              boolean loadSSTables)
+                              boolean loadSSTables,
+                              boolean registerBookkeeping)
     {
         assert metadata != null : "null metadata for " + keyspace + ":" + columnFamilyName;
 
@@ -315,7 +335,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         logger.info("Initializing {}.{}", keyspace.getName(), name);
 
         // scan for sstables corresponding to this cf and load them
-        data = new DataTracker(this, loadSSTables);
+        data = new Tracker(this, loadSSTables);
 
         if (data.loadsstables)
         {
@@ -343,46 +363,59 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 indexManager.addIndexedColumn(info);
         }
 
-        // register the mbean
-        String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
-        mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
-        try
-        {
-            MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-            ObjectName nameObj = new ObjectName(mbeanName);
-            mbs.registerMBean(this, nameObj);
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-        logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
-        latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
+        if (registerBookkeeping)
         {
-            public void run()
+            // register the mbean
+            String type = this.partitioner instanceof LocalPartitioner ? "IndexColumnFamilies" : "ColumnFamilies";
+            mbeanName = "org.apache.cassandra.db:type=" + type + ",keyspace=" + this.keyspace.getName() + ",columnfamily=" + name;
+            try
+            {
+                MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+                ObjectName nameObj = new ObjectName(mbeanName);
+                mbs.registerMBean(this, nameObj);
+            }
+            catch (Exception e)
             {
-                SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
-                switch (retryPolicy.type)
+                throw new RuntimeException(e);
+            }
+            logger.debug("retryPolicy for {} is {}", name, this.metadata.getSpeculativeRetry());
+            latencyCalculator = ScheduledExecutors.optionalTasks.scheduleWithFixedDelay(new Runnable()
+            {
+                public void run()
                 {
-                    case PERCENTILE:
-                        // get percentile in nanos
-                        sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value) * 1000d);
-                        break;
-                    case CUSTOM:
-                        // convert to nanos, since configuration is in millisecond
-                        sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
-                        break;
-                    default:
-                        sampleLatencyNanos = Long.MAX_VALUE;
-                        break;
+                    SpeculativeRetry retryPolicy = ColumnFamilyStore.this.metadata.getSpeculativeRetry();
+                    switch (retryPolicy.type)
+                    {
+                        case PERCENTILE:
+                            // get percentile in nanos
+                            sampleLatencyNanos = (long) (metric.coordinatorReadLatency.getSnapshot().getValue(retryPolicy.value) * 1000d);
+                            break;
+                        case CUSTOM:
+                            // convert to nanos, since configuration is in millisecond
+                            sampleLatencyNanos = (long) (retryPolicy.value * 1000d * 1000d);
+                            break;
+                        default:
+                            sampleLatencyNanos = Long.MAX_VALUE;
+                            break;
+                    }
                 }
-            }
-        }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+            }, DatabaseDescriptor.getReadRpcTimeout(), DatabaseDescriptor.getReadRpcTimeout(), TimeUnit.MILLISECONDS);
+        }
+        else
+        {
+            latencyCalculator = ScheduledExecutors.optionalTasks.schedule(Runnables.doNothing(), 0, TimeUnit.NANOSECONDS);
+            mbeanName = null;
+        }
     }
 
     /** call when dropping or renaming a CF. Performs mbean housekeeping and invalidates CFS to other operations */
     public void invalidate()
     {
+        invalidate(true);
+    }
+
+    public void invalidate(boolean expectMBean)
+    {
         // disable and cancel in-progress compactions before invalidating
         valid = false;
 
@@ -392,21 +425,24 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
         catch (Exception e)
         {
-            JVMStabilityInspector.inspectThrowable(e);
-            // this shouldn't block anything.
-            logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+            if (expectMBean)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+                // this shouldn't block anything.
+                logger.warn("Failed unregistering mbean: {}", mbeanName, e);
+            }
         }
 
         latencyCalculator.cancel(false);
         SystemKeyspace.removeTruncationRecord(metadata.cfId);
-        data.unreferenceSSTables();
+        data.dropSSTables();
         indexManager.invalidate();
 
         invalidateCaches();
     }
 
     /**
-     * Removes every SSTable in the directory from the DataTracker's view.
+     * Removes every SSTable in the directory from the Tracker's view.
      * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
      */
     void maybeRemoveUnreadableSSTables(File directory)
@@ -542,7 +578,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     }
 
     /**
-     * Replacing compacted sstables is atomic as far as observers of DataTracker are concerned, but not on the
+     * Replacing compacted sstables is atomic as far as observers of Tracker are concerned, but not on the
      * filesystem: first the new sstables are renamed to "live" status (i.e., the tmp marker is removed), then
      * their ancestors are removed.
      *
@@ -826,7 +862,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     /*
      * switchMemtable puts Memtable.getSortedContents on the writer executor.  When the write is complete,
      * we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
-     * This method does not block except for synchronizing on DataTracker, but the Future it returns will
+     * This method does not block except for synchronizing on Tracker, but the Future it returns will
      * not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
      * marked clean up to the position owned by the Memtable.
      */
@@ -849,7 +885,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         // reclaiming includes that which we are GC-ing;
         float onHeapRatio = 0, offHeapRatio = 0;
         long onHeapTotal = 0, offHeapTotal = 0;
-        Memtable memtable = getDataTracker().getView().getCurrentMemtable();
+        Memtable memtable = getTracker().getView().getCurrentMemtable();
         onHeapRatio +=  memtable.getAllocator().onHeap().ownershipRatio();
         offHeapRatio += memtable.getAllocator().offHeap().ownershipRatio();
         onHeapTotal += memtable.getAllocator().onHeap().owns();
@@ -859,7 +895,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         {
             if (index.getIndexCfs() != null)
             {
-                MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
                 onHeapRatio += allocator.onHeap().ownershipRatio();
                 offHeapRatio += allocator.offHeap().ownershipRatio();
                 onHeapTotal += allocator.onHeap().owns();
@@ -984,7 +1020,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     }
 
     /**
-     * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the DataTracker monitor.
+     * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the Tracker monitor.
      * In the constructor the current memtable(s) are swapped, and a barrier on outstanding writes is issued;
      * when run by the flushWriter the barrier is waited on to ensure all outstanding writes have completed
      * before all memtables are immediately written, and the CL is either immediately marked clean or, if
@@ -1117,7 +1153,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 // we take a reference to the current main memtable for the CF prior to snapping its ownership ratios
                 // to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only
                 // swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them
-                Memtable current = cfs.getDataTracker().getView().getCurrentMemtable();
+                Memtable current = cfs.getTracker().getView().getCurrentMemtable();
 
                 // find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF,
                 // both on- and off-heap, and select the largest of the two ratios to weight this CF
@@ -1129,7 +1165,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 {
                     if (index.getIndexCfs() != null)
                     {
-                        MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                        MemtableAllocator allocator = index.getIndexCfs().getTracker().getView().getCurrentMemtable().getAllocator();
                         onHeap += allocator.onHeap().ownershipRatio();
                         offHeap += allocator.offHeap().ownershipRatio();
                     }
@@ -1278,7 +1314,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (!sstables.iterator().hasNext())
             return ImmutableSet.of();
 
-        DataTracker.SSTableIntervalTree tree = data.getView().intervalTree;
+        SSTableIntervalTree tree = data.getView().intervalTree;
 
         Set<SSTableReader> results = null;
         for (SSTableReader sstable : sstables)
@@ -1454,7 +1490,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
     {
         assert !sstables.isEmpty();
-        data.markObsolete(sstables, compactionType);
+        maybeFail(data.dropSSTables(Predicates.in(sstables), compactionType, null));
     }
 
     void replaceFlushed(Memtable memtable, SSTableReader sstable)
@@ -1473,7 +1509,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     /**
      * Package protected for access from the CompactionManager.
      */
-    public DataTracker getDataTracker()
+    public Tracker getTracker()
     {
         return data;
     }
@@ -1485,7 +1521,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public Set<SSTableReader> getUncompactingSSTables()
     {
-        return data.getUncompactingSSTables();
+        return data.getUncompacting();
     }
 
     public ColumnFamily getColumnFamily(DecoratedKey key,
@@ -1759,7 +1795,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return repairedSSTables;
     }
 
-    public RefViewFragment selectAndReference(Function<DataTracker.View, List<SSTableReader>> filter)
+    public RefViewFragment selectAndReference(Function<View, List<SSTableReader>> filter)
     {
         while (true)
         {
@@ -1770,9 +1806,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    public ViewFragment select(Function<DataTracker.View, List<SSTableReader>> filter)
+    public ViewFragment select(Function<View, List<SSTableReader>> filter)
     {
-        DataTracker.View view = data.getView();
+        View view = data.getView();
         List<SSTableReader> sstables = view.intervalTree.isEmpty()
                                        ? Collections.<SSTableReader>emptyList()
                                        : filter.apply(view);
@@ -1784,12 +1820,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return a ViewFragment containing the sstables and memtables that may need to be merged
      * for the given @param key, according to the interval tree
      */
-    public Function<DataTracker.View, List<SSTableReader>> viewFilter(final DecoratedKey key)
+    public Function<View, List<SSTableReader>> viewFilter(final DecoratedKey key)
     {
         assert !key.isMinimum();
-        return new Function<DataTracker.View, List<SSTableReader>>()
+        return new Function<View, List<SSTableReader>>()
         {
-            public List<SSTableReader> apply(DataTracker.View view)
+            public List<SSTableReader> apply(View view)
             {
                 return compactionStrategyWrapper.filterSSTablesForReads(view.intervalTree.search(key));
             }
@@ -1800,17 +1836,43 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * @return a ViewFragment containing the sstables and memtables that may need to be merged
      * for rows within @param rowBounds, inclusive, according to the interval tree.
      */
-    public Function<DataTracker.View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
+    public Function<View, List<SSTableReader>> viewFilter(final AbstractBounds<RowPosition> rowBounds)
     {
-        return new Function<DataTracker.View, List<SSTableReader>>()
+        return new Function<View, List<SSTableReader>>()
         {
-            public List<SSTableReader> apply(DataTracker.View view)
+            public List<SSTableReader> apply(View view)
             {
                 return compactionStrategyWrapper.filterSSTablesForReads(view.sstablesInBounds(rowBounds));
             }
         };
     }
 
+    /**
+     * @return a ViewFragment containing the sstables and memtables that may need to be merged
+     * for rows for all of @param rowBoundsCollection, inclusive, according to the interval tree.
+     */
+    public Function<View, List<SSTableReader>> viewFilter(final Collection<AbstractBounds<RowPosition>> rowBoundsCollection, final boolean includeRepaired)
+    {
+        return new Function<View, List<SSTableReader>>()
+        {
+            public List<SSTableReader> apply(View view)
+            {
+                Set<SSTableReader> sstables = Sets.newHashSet();
+                for (AbstractBounds<RowPosition> rowBounds : rowBoundsCollection)
+                {
+                    for (SSTableReader sstable : view.sstablesInBounds(rowBounds))
+                    {
+                        if (includeRepaired || !sstable.isRepaired())
+                            sstables.add(sstable);
+                    }
+                }
+
+                logger.debug("ViewFilter for {}/{} sstables", sstables.size(), getSSTables().size());
+                return ImmutableList.copyOf(sstables);
+            }
+        };
+    }
+
     public List<String> getSSTablesForKey(String key)
     {
         DecoratedKey dk = partitioner.decorateKey(metadata.getKeyValidator().fromString(key));
@@ -2388,6 +2450,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * thread safety.  All we do is wipe the sstable containers clean, while leaving the actual
      * data files present on disk.  (This allows tests to easily call loadNewSSTables on them.)
      */
+    @VisibleForTesting
     public void clearUnsafe()
     {
         for (final ColumnFamilyStore cfs : concatWithIndexes())
@@ -2396,7 +2459,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 public Void call()
                 {
-                    cfs.data.init();
+                    cfs.data.reset();
                     return null;
                 }
             }, true);
@@ -2489,7 +2552,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 // doublecheck that we finished, instead of timing out
                 for (ColumnFamilyStore cfs : selfWithIndexes)
                 {
-                    if (!cfs.getDataTracker().getCompacting().isEmpty())
+                    if (!cfs.getTracker().getCompacting().isEmpty())
                     {
                         logger.warn("Unable to cancel in-progress compactions for {}.  Perhaps there is an unusually large row in progress somewhere, or the system is simply overloaded.", metadata.cfName);
                         return null;
@@ -2515,19 +2578,19 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    public Iterable<SSTableReader> markAllCompacting()
+    public LifecycleTransaction markAllCompacting(final OperationType operationType)
     {
-        Callable<Iterable<SSTableReader>> callable = new Callable<Iterable<SSTableReader>>()
+        Callable<LifecycleTransaction> callable = new Callable<LifecycleTransaction>()
         {
-            public Iterable<SSTableReader> call() throws Exception
+            public LifecycleTransaction call() throws Exception
             {
                 assert data.getCompacting().isEmpty() : data.getCompacting();
                 Collection<SSTableReader> sstables = Lists.newArrayList(AbstractCompactionStrategy.filterSuspectSSTables(getSSTables()));
                 if (Iterables.isEmpty(sstables))
-                    return Collections.emptyList();
-                boolean success = data.markCompacting(sstables);
-                assert success : "something marked things compacting while compactions are disabled";
-                return sstables;
+                    return null;
+                LifecycleTransaction modifier = data.tryModify(sstables, operationType);
+                assert modifier != null: "something marked things compacting while compactions are disabled";
+                return modifier;
             }
         };
 
@@ -2634,12 +2697,23 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public int getMeanColumns()
     {
-        return data.getMeanColumns();
+        long sum = 0;
+        long count = 0;
+        for (SSTableReader sstable : getSSTables())
+        {
+            long n = sstable.getEstimatedColumnCount().count();
+            sum += sstable.getEstimatedColumnCount().mean() * n;
+            count += n;
+        }
+        return count > 0 ? (int) (sum / count) : 0;
     }
 
     public long estimateKeys()
     {
-        return data.estimatedKeys();
+        long n = 0;
+        for (SSTableReader sstable : getSSTables())
+            n += sstable.estimatedKeys();
+        return n;
     }
 
     /** true if this CFS contains secondary index data */
@@ -2703,18 +2777,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    /**
-     * Returns the creation time of the oldest memtable not fully flushed yet.
-     */
-    public long oldestUnflushedMemtable()
-    {
-        return data.getView().getOldestMemtable().creationTime();
-    }
-
     public boolean isEmpty()
     {
-        DataTracker.View view = data.getView();
-        return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.getCurrentMemtable() == view.getOldestMemtable();
+        View view = data.getView();
+        return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.liveMemtables.size() <= 1 && view.flushingMemtables.size() == 0;
     }
 
     private boolean isRowCacheEnabled()
@@ -2753,7 +2819,16 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public double getDroppableTombstoneRatio()
     {
-        return getDataTracker().getDroppableTombstoneRatio();
+        double allDroppable = 0;
+        long allColumns = 0;
+        int localTime = (int)(System.currentTimeMillis()/1000);
+
+        for (SSTableReader sstable : getSSTables())
+        {
+            allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
+            allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
+        }
+        return allColumns > 0 ? allDroppable / allColumns : 0;
     }
 
     public long trueSnapshotsSize()
@@ -2770,9 +2845,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     // returns the "canonical" version of any current sstable, i.e. if an sstable is being replaced and is only partially
     // visible to reads, this sstable will be returned as its original entirety, and its replacement will not be returned
     // (even if it completely replaces it)
-    public static final Function<DataTracker.View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+    public static final Function<View, List<SSTableReader>> CANONICAL_SSTABLES = new Function<View, List<SSTableReader>>()
     {
-        public List<SSTableReader> apply(DataTracker.View view)
+        public List<SSTableReader> apply(View view)
         {
             List<SSTableReader> sstables = new ArrayList<>();
             for (SSTableReader sstable : view.compacting)
@@ -2785,9 +2860,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     };
 
-    public static final Function<DataTracker.View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<DataTracker.View, List<SSTableReader>>()
+    public static final Function<View, List<SSTableReader>> UNREPAIRED_SSTABLES = new Function<View, List<SSTableReader>>()
     {
-        public List<SSTableReader> apply(DataTracker.View view)
+        public List<SSTableReader> apply(View view)
         {
             List<SSTableReader> sstables = new ArrayList<>();
             for (SSTableReader sstable : CANONICAL_SSTABLES.apply(view))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
deleted file mode 100644
index 36f22c5..0000000
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ /dev/null
@@ -1,793 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.File;
-import java.util.*;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.*;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.metrics.StorageMetrics;
-import org.apache.cassandra.notifications.*;
-import org.apache.cassandra.utils.Interval;
-import org.apache.cassandra.utils.IntervalTree;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.concurrent.Refs;
-
-public class DataTracker
-{
-    private static final Logger logger = LoggerFactory.getLogger(DataTracker.class);
-
-    public final Collection<INotificationConsumer> subscribers = new CopyOnWriteArrayList<>();
-    public final ColumnFamilyStore cfstore;
-    private final AtomicReference<View> view;
-
-    // Indicates if it is safe to load the initial sstables (may not be true when running in
-    //standalone processes meant to repair or upgrade sstables (e.g. standalone scrubber)
-    public final boolean loadsstables;
-
-    public DataTracker(ColumnFamilyStore cfstore, boolean loadsstables)
-    {
-        this.cfstore = cfstore;
-        this.view = new AtomicReference<>();
-        this.loadsstables = loadsstables;
-        this.init();
-    }
-
-    // get the Memtable that the ordered writeOp should be directed to
-    public Memtable getMemtableFor(OpOrder.Group opGroup, ReplayPosition replayPosition)
-    {
-        // since any new memtables appended to the list after we fetch it will be for operations started
-        // after us, we can safely assume that we will always find the memtable that 'accepts' us;
-        // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
-
-        // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
-        // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
-        // assign operations to a memtable that was retired/queued before we started)
-        for (Memtable memtable : view.get().liveMemtables)
-        {
-            if (memtable.accepts(opGroup, replayPosition))
-                return memtable;
-        }
-        throw new AssertionError(view.get().liveMemtables.toString());
-    }
-
-    public Set<SSTableReader> getSSTables()
-    {
-        return view.get().sstables;
-    }
-
-    public Set<SSTableReader> getUncompactingSSTables()
-    {
-        return view.get().nonCompactingSStables();
-    }
-
-    public Iterable<SSTableReader> getUncompactingSSTables(Iterable<SSTableReader> candidates)
-    {
-        final View v = view.get();
-        return Iterables.filter(candidates, new Predicate<SSTableReader>()
-        {
-            public boolean apply(SSTableReader sstable)
-            {
-                return !v.compacting.contains(sstable);
-            }
-        });
-    }
-
-    public View getView()
-    {
-        return view.get();
-    }
-
-    /**
-     * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
-     * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
-     * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
-     * must be followed by discarding(m), they cannot be interleaved.
-     *
-     * @return the previously active memtable
-     */
-    public Memtable switchMemtable(boolean truncating)
-    {
-        Memtable newMemtable = new Memtable(cfstore);
-        Memtable toFlushMemtable;
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            toFlushMemtable = currentView.getCurrentMemtable();
-            newView = currentView.switchMemtable(newMemtable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (truncating)
-            notifyRenewed(newMemtable);
-
-        return toFlushMemtable;
-    }
-
-    public void markFlushing(Memtable memtable)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.markFlushing(memtable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-    }
-
-    public void replaceFlushed(Memtable memtable, SSTableReader sstable)
-    {
-        // sstable may be null if we flushed batchlog and nothing needed to be retained
-
-        if (!cfstore.isValid())
-        {
-            View currentView, newView;
-            do
-            {
-                currentView = view.get();
-                newView = currentView.replaceFlushed(memtable, sstable);
-                if (sstable != null)
-                    newView = newView.replace(Arrays.asList(sstable), Collections.<SSTableReader>emptyList());
-            }
-            while (!view.compareAndSet(currentView, newView));
-            return;
-        }
-
-        // back up before creating a new View (which makes the new one eligible for compaction)
-        if (sstable != null)
-            maybeIncrementallyBackup(sstable);
-
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replaceFlushed(memtable, sstable);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (sstable != null)
-        {
-            addNewSSTablesSize(Arrays.asList(sstable));
-            notifyAdded(sstable);
-        }
-    }
-
-    public void maybeIncrementallyBackup(final SSTableReader sstable)
-    {
-        if (!DatabaseDescriptor.isIncrementalBackupsEnabled())
-            return;
-
-        File backupsDir = Directories.getBackupsDirectory(sstable.descriptor);
-        sstable.createLinks(FileUtils.getCanonicalPath(backupsDir));
-    }
-
-    /**
-     * @return true if we are able to mark the given @param sstables as compacted, before anyone else
-     *
-     * Note that we could acquire references on the marked sstables and release them in
-     * unmarkCompacting, but since we will never call markObsolete on a sstable marked
-     * as compacting (unless there is a serious bug), we can skip this.
-     */
-    public boolean markCompacting(Iterable<SSTableReader> sstables)
-    {
-        return markCompacting(sstables, false, false);
-    }
-
-    public boolean markCompacting(Iterable<SSTableReader> sstables, boolean newTables, boolean offline)
-    {
-        assert sstables != null && !Iterables.isEmpty(sstables);
-        while (true)
-        {
-            final View currentView = view.get();
-            if (Iterables.any(sstables, Predicates.in(currentView.compacting)))
-                return false;
-
-            Predicate<SSTableReader> live = new Predicate<SSTableReader>()
-            {
-                public boolean apply(SSTableReader sstable)
-                {
-                    return currentView.sstablesMap.get(sstable) == sstable && !sstable.isMarkedCompacted();
-                }
-            };
-            if (newTables)
-                assert !Iterables.any(sstables, Predicates.in(currentView.sstables));
-            else if (!offline && !Iterables.all(sstables, live))
-                return false;
-
-            View newView = currentView.markCompacting(sstables);
-            if (view.compareAndSet(currentView, newView))
-                return true;
-        }
-    }
-
-    /**
-     * Removes files from compacting status: this is different from 'markObsolete'
-     * because it should be run regardless of whether a compaction succeeded.
-     */
-    public void unmarkCompacting(Iterable<SSTableReader> unmark)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.unmarkCompacting(unmark);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (!cfstore.isValid())
-        {
-            // when the CFS is invalidated, it will call unreferenceSSTables().  However, unreferenceSSTables only deals
-            // with sstables that aren't currently being compacted.  If there are ongoing compactions that finish or are
-            // interrupted after the CFS is invalidated, those sstables need to be unreferenced as well, so we do that here.
-            unreferenceSSTables();
-        }
-    }
-
-    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
-    {
-        removeSSTablesFromTracker(sstables);
-        releaseReferences(sstables, false);
-        notifySSTablesChanged(sstables, Collections.<SSTableReader>emptyList(), compactionType);
-    }
-
-    /**
-     *
-     * @param oldSSTables
-     * @param allReplacements
-     * @param compactionType
-     */
-    // note that this DOES NOT insert the replacement sstables, it only removes the old sstables and notifies any listeners
-    // that they have been replaced by the provided sstables, which must have been performed by an earlier replaceReaders() call
-    public void markCompactedSSTablesReplaced(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> allReplacements, OperationType compactionType)
-    {
-        removeSSTablesFromTracker(oldSSTables);
-        releaseReferences(oldSSTables, false);
-        notifySSTablesChanged(oldSSTables, allReplacements, compactionType);
-        addNewSSTablesSize(allReplacements);
-    }
-
-    public void addInitialSSTables(Collection<SSTableReader> sstables)
-    {
-        addSSTablesToTracker(sstables);
-        // no notifications or backup necessary
-    }
-
-    public void addSSTables(Collection<SSTableReader> sstables)
-    {
-        addSSTablesToTracker(sstables);
-        for (SSTableReader sstable : sstables)
-        {
-            maybeIncrementallyBackup(sstable);
-            notifyAdded(sstable);
-        }
-    }
-
-    /**
-     * Replaces existing sstables with new instances, makes sure compaction strategies have the correct instance
-     *
-     * @param toReplace
-     * @param replaceWith
-     */
-    public void replaceWithNewInstances(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
-    {
-        replaceReaders(toReplace, replaceWith, true);
-    }
-
-    /**
-     * Adds the early opened files to the data tracker, but does not tell compaction strategies about it
-     *
-     * note that we dont track the live size of these sstables
-     * @param toReplace
-     * @param replaceWith
-     */
-    public void replaceEarlyOpenedFiles(Collection<SSTableReader> toReplace, Collection<SSTableReader> replaceWith)
-    {
-        for (SSTableReader s : toReplace)
-            assert s.openReason == SSTableReader.OpenReason.EARLY;
-        // note that we can replace an early opened file with a real one
-        replaceReaders(toReplace, replaceWith, false);
-    }
-
-    /**
-     * removes all sstables that are not busy compacting.
-     */
-    public void unreferenceSSTables()
-    {
-        Set<SSTableReader> notCompacting;
-
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            if (!currentView.compacting.isEmpty())
-                logger.error("Set of compacting sstables is non-empty when invalidating sstables {}", currentView.compacting);
-            notCompacting = currentView.nonCompactingSStables();
-            newView = currentView.replace(notCompacting, Collections.<SSTableReader>emptySet());
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (notCompacting.isEmpty())
-        {
-            // notifySSTablesChanged -> LeveledManifest.promote doesn't like a no-op "promotion"
-            return;
-        }
-        notifySSTablesChanged(notCompacting, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
-        removeOldSSTablesSize(notCompacting);
-        releaseReferences(notCompacting, true);
-    }
-
-    /**
-     * Removes every SSTable in the directory from the DataTracker's view.
-     * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
-     */
-    void removeUnreadableSSTables(File directory)
-    {
-        View currentView, newView;
-        Set<SSTableReader> remaining = new HashSet<>();
-        do
-        {
-            currentView = view.get();
-            for (SSTableReader r : currentView.nonCompactingSStables())
-                if (!r.descriptor.directory.equals(directory))
-                    remaining.add(r);
-
-            if (remaining.size() == currentView.nonCompactingSStables().size())
-                return;
-
-            newView = currentView.replace(currentView.sstables, remaining);
-        }
-        while (!view.compareAndSet(currentView, newView));
-        for (SSTableReader sstable : currentView.sstables)
-            if (!remaining.contains(sstable))
-                sstable.selfRef().release();
-        notifySSTablesChanged(remaining, Collections.<SSTableReader>emptySet(), OperationType.UNKNOWN);
-    }
-
-    /** (Re)initializes the tracker, purging all references. */
-    void init()
-    {
-        view.set(new View(
-                         ImmutableList.of(new Memtable(cfstore)),
-                         ImmutableList.<Memtable>of(),
-                         Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
-                         Collections.<SSTableReader>emptySet(),
-                         SSTableIntervalTree.empty()));
-    }
-
-    /**
-     * A special kind of replacement for SSTableReaders that were cloned with a new index summary sampling level (see
-     * SSTableReader.cloneWithNewSummarySamplingLevel and CASSANDRA-5519).  This does not mark the old reader
-     * as compacted.
-     * @param oldSSTables replaced readers
-     * @param newSSTables replacement readers
-     */
-    private void replaceReaders(Collection<SSTableReader> oldSSTables, Collection<SSTableReader> newSSTables, boolean notify)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(oldSSTables, newSSTables);
-        }
-        while (!view.compareAndSet(currentView, newView));
-
-        if (!oldSSTables.isEmpty() && notify)
-            notifySSTablesChanged(oldSSTables, newSSTables, OperationType.UNKNOWN);
-
-        for (SSTableReader sstable : newSSTables)
-            sstable.setTrackedBy(this);
-
-        Refs.release(Refs.selfRefs(oldSSTables));
-    }
-
-    private void removeSSTablesFromTracker(Collection<SSTableReader> oldSSTables)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(oldSSTables, Collections.<SSTableReader>emptyList());
-        }
-        while (!view.compareAndSet(currentView, newView));
-        removeOldSSTablesSize(oldSSTables);
-    }
-
-    private void addSSTablesToTracker(Collection<SSTableReader> sstables)
-    {
-        View currentView, newView;
-        do
-        {
-            currentView = view.get();
-            newView = currentView.replace(Collections.<SSTableReader>emptyList(), sstables);
-        }
-        while (!view.compareAndSet(currentView, newView));
-        addNewSSTablesSize(sstables);
-    }
-
-    private void addNewSSTablesSize(Iterable<SSTableReader> newSSTables)
-    {
-        for (SSTableReader sstable : newSSTables)
-        {
-            if (logger.isDebugEnabled())
-                logger.debug(String.format("adding %s to list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
-            long size = sstable.bytesOnDisk();
-            StorageMetrics.load.inc(size);
-            cfstore.metric.liveDiskSpaceUsed.inc(size);
-            cfstore.metric.totalDiskSpaceUsed.inc(size);
-            sstable.setTrackedBy(this);
-        }
-    }
-
-    private void removeOldSSTablesSize(Iterable<SSTableReader> oldSSTables)
-    {
-        for (SSTableReader sstable : oldSSTables)
-        {
-            if (logger.isDebugEnabled())
-                logger.debug(String.format("removing %s from list of files tracked for %s.%s",
-                            sstable.descriptor, cfstore.keyspace.getName(), cfstore.name));
-            long size = sstable.bytesOnDisk();
-            StorageMetrics.load.dec(size);
-            cfstore.metric.liveDiskSpaceUsed.dec(size);
-        }
-    }
-
-    private void releaseReferences(Iterable<SSTableReader> oldSSTables, boolean tolerateCompacted)
-    {
-        for (SSTableReader sstable : oldSSTables)
-        {
-            boolean firstToCompact = sstable.markObsolete();
-            assert tolerateCompacted || firstToCompact : sstable + " was already marked compacted";
-            sstable.selfRef().release();
-        }
-    }
-
-    public void spaceReclaimed(long size)
-    {
-        cfstore.metric.totalDiskSpaceUsed.dec(size);
-    }
-
-    public long estimatedKeys()
-    {
-        long n = 0;
-        for (SSTableReader sstable : getSSTables())
-            n += sstable.estimatedKeys();
-        return n;
-    }
-
-    public int getMeanColumns()
-    {
-        long sum = 0;
-        long count = 0;
-        for (SSTableReader sstable : getSSTables())
-        {
-            long n = sstable.getEstimatedColumnCount().count();
-            sum += sstable.getEstimatedColumnCount().mean() * n;
-            count += n;
-        }
-        return count > 0 ? (int) (sum / count) : 0;
-    }
-
-    public double getDroppableTombstoneRatio()
-    {
-        double allDroppable = 0;
-        long allColumns = 0;
-        int localTime = (int)(System.currentTimeMillis()/1000);
-
-        for (SSTableReader sstable : getSSTables())
-        {
-            allDroppable += sstable.getDroppableTombstonesBefore(localTime - sstable.metadata.getGcGraceSeconds());
-            allColumns += sstable.getEstimatedColumnCount().mean() * sstable.getEstimatedColumnCount().count();
-        }
-        return allColumns > 0 ? allDroppable / allColumns : 0;
-    }
-
-    public void notifySSTablesChanged(Collection<SSTableReader> removed, Collection<SSTableReader> added, OperationType compactionType)
-    {
-        INotification notification = new SSTableListChangedNotification(added, removed, compactionType);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyAdded(SSTableReader added)
-    {
-        INotification notification = new SSTableAddedNotification(added);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifySSTableRepairedStatusChanged(Collection<SSTableReader> repairStatusesChanged)
-    {
-        INotification notification = new SSTableRepairStatusChanged(repairStatusesChanged);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-
-    }
-
-    public void notifyDeleting(SSTableReader deleting)
-    {
-        INotification notification = new SSTableDeletingNotification(deleting);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyRenewed(Memtable renewed)
-    {
-        INotification notification = new MemtableRenewedNotification(renewed);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void notifyTruncated(long truncatedAt)
-    {
-        INotification notification = new TruncationNotification(truncatedAt);
-        for (INotificationConsumer subscriber : subscribers)
-            subscriber.handleNotification(notification, this);
-    }
-
-    public void subscribe(INotificationConsumer consumer)
-    {
-        subscribers.add(consumer);
-    }
-
-    public void unsubscribe(INotificationConsumer consumer)
-    {
-        subscribers.remove(consumer);
-    }
-
-    public static SSTableIntervalTree buildIntervalTree(Iterable<SSTableReader> sstables)
-    {
-        return new SSTableIntervalTree(buildIntervals(sstables));
-    }
-
-    public static List<Interval<RowPosition, SSTableReader>> buildIntervals(Iterable<SSTableReader> sstables)
-    {
-        List<Interval<RowPosition, SSTableReader>> intervals = new ArrayList<>(Iterables.size(sstables));
-        for (SSTableReader sstable : sstables)
-            intervals.add(Interval.<RowPosition, SSTableReader>create(sstable.first, sstable.last, sstable));
-        return intervals;
-    }
-
-    public Set<SSTableReader> getCompacting()
-    {
-        return getView().compacting;
-    }
-
-    public static class SSTableIntervalTree extends IntervalTree<RowPosition, SSTableReader, Interval<RowPosition, SSTableReader>>
-    {
-        private static final SSTableIntervalTree EMPTY = new SSTableIntervalTree(null);
-
-        private SSTableIntervalTree(Collection<Interval<RowPosition, SSTableReader>> intervals)
-        {
-            super(intervals);
-        }
-
-        public static SSTableIntervalTree empty()
-        {
-            return EMPTY;
-        }
-    }
-
-    /**
-     * An immutable structure holding the current memtable, the memtables pending
-     * flush, the sstables for a column family, and the sstables that are active
-     * in compaction (a subset of the sstables).
-     */
-    public static class View
-    {
-        /**
-         * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
-         * and the new replacement memtable, until all outstanding write operations on the old table complete.
-         * The last item in the list is always the "current" memtable.
-         */
-        private final List<Memtable> liveMemtables;
-        /**
-         * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
-         * flushed. In chronologically ascending order.
-         */
-        private final List<Memtable> flushingMemtables;
-        public final Set<SSTableReader> compacting;
-        public final Set<SSTableReader> sstables;
-        // we use a Map here so that we can easily perform identity checks as well as equality checks.
-        // When marking compacting, we now  indicate if we expect the sstables to be present (by default we do),
-        // and we then check that not only are they all present in the live set, but that the exact instance present is
-        // the one we made our decision to compact against.
-        public final Map<SSTableReader, SSTableReader> sstablesMap;
-
-        // all sstables that are still in the live set, but have been completely shadowed by a replacement sstable
-        public final Set<SSTableReader> shadowed;
-        public final SSTableIntervalTree intervalTree;
-
-        View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Set<SSTableReader> compacting, Set<SSTableReader> shadowed, SSTableIntervalTree intervalTree)
-        {
-            this.shadowed = shadowed;
-            assert liveMemtables != null;
-            assert flushingMemtables != null;
-            assert sstables != null;
-            assert compacting != null;
-            assert intervalTree != null;
-
-            this.liveMemtables = liveMemtables;
-            this.flushingMemtables = flushingMemtables;
-
-            this.sstablesMap = sstables;
-            this.sstables = sstablesMap.keySet();
-            this.compacting = compacting;
-            this.intervalTree = intervalTree;
-        }
-
-        public Memtable getOldestMemtable()
-        {
-            if (!flushingMemtables.isEmpty())
-                return flushingMemtables.get(0);
-            return liveMemtables.get(0);
-        }
-
-        public Memtable getCurrentMemtable()
-        {
-            return liveMemtables.get(liveMemtables.size() - 1);
-        }
-
-        public Iterable<Memtable> getMemtablesPendingFlush()
-        {
-            if (liveMemtables.size() == 1)
-                return flushingMemtables;
-            return Iterables.concat(liveMemtables.subList(0, 1), flushingMemtables);
-        }
-
-        /**
-         * @return the active memtable and all the memtables that are pending flush.
-         */
-        public Iterable<Memtable> getAllMemtables()
-        {
-            return Iterables.concat(flushingMemtables, liveMemtables);
-        }
-
-        public Sets.SetView<SSTableReader> nonCompactingSStables()
-        {
-            return Sets.difference(ImmutableSet.copyOf(sstables), compacting);
-        }
-
-        View switchMemtable(Memtable newMemtable)
-        {
-            List<Memtable> newLiveMemtables = ImmutableList.<Memtable>builder().addAll(liveMemtables).add(newMemtable).build();
-            return new View(newLiveMemtables, flushingMemtables, sstablesMap, compacting, shadowed, intervalTree);
-        }
-
-        View markFlushing(Memtable toFlushMemtable)
-        {
-            List<Memtable> live = liveMemtables, flushing = flushingMemtables;
-
-            // since we can have multiple flushes queued, we may occasionally race and start a flush out of order,
-            // so must locate it in the list to remove, rather than just removing from the beginning
-            int i = live.indexOf(toFlushMemtable);
-            assert i < live.size() - 1;
-            List<Memtable> newLive = ImmutableList.<Memtable>builder()
-                                                  .addAll(live.subList(0, i))
-                                                  .addAll(live.subList(i + 1, live.size()))
-                                                  .build();
-
-            // similarly, if we out-of-order markFlushing once, we may afterwards need to insert a memtable into the
-            // flushing list in a position other than the end, though this will be rare
-            i = flushing.size();
-            while (i > 0 && flushing.get(i - 1).creationTime() > toFlushMemtable.creationTime())
-                i--;
-            List<Memtable> newFlushing = ImmutableList.<Memtable>builder()
-                                                      .addAll(flushing.subList(0, i))
-                                                      .add(toFlushMemtable)
-                                                      .addAll(flushing.subList(i, flushing.size()))
-                                                      .build();
-
-            return new View(newLive, newFlushing, sstablesMap, compacting, shadowed, intervalTree);
-        }
-
-        View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
-        {
-            int index = flushingMemtables.indexOf(flushedMemtable);
-            List<Memtable> newQueuedMemtables = ImmutableList.<Memtable>builder()
-                                                             .addAll(flushingMemtables.subList(0, index))
-                                                             .addAll(flushingMemtables.subList(index + 1, flushingMemtables.size()))
-                                                             .build();
-            Map<SSTableReader, SSTableReader> newSSTables = sstablesMap;
-            SSTableIntervalTree intervalTree = this.intervalTree;
-            if (newSSTable != null)
-            {
-                assert !sstables.contains(newSSTable);
-                assert !shadowed.contains(newSSTable);
-                newSSTables = ImmutableMap.<SSTableReader, SSTableReader>builder()
-                                          .putAll(sstablesMap).put(newSSTable, newSSTable).build();
-                intervalTree = buildIntervalTree(newSSTables.keySet());
-            }
-            return new View(liveMemtables, newQueuedMemtables, newSSTables, compacting, shadowed, intervalTree);
-        }
-
-        View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
-        {
-            ImmutableSet<SSTableReader> oldSet = ImmutableSet.copyOf(oldSSTables);
-            int newSSTablesSize = shadowed.size() + sstables.size() - oldSSTables.size() + Iterables.size(replacements);
-            assert newSSTablesSize >= Iterables.size(replacements) : String.format("Incoherent new size %d replacing %s by %s in %s", newSSTablesSize, oldSSTables, replacements, this);
-            Map<SSTableReader, SSTableReader> newSSTables = new HashMap<>(newSSTablesSize);
-            Set<SSTableReader> newShadowed = new HashSet<>(shadowed.size());
-
-            for (SSTableReader sstable : sstables)
-                if (!oldSet.contains(sstable))
-                    newSSTables.put(sstable, sstable);
-
-            for (SSTableReader sstable : shadowed)
-                if (!oldSet.contains(sstable))
-                    newShadowed.add(sstable);
-
-            for (SSTableReader replacement : replacements)
-            {
-                if (replacement.openReason == SSTableReader.OpenReason.SHADOWED)
-                    newShadowed.add(replacement);
-                else
-                    newSSTables.put(replacement, replacement);
-            }
-
-            assert newSSTables.size() + newShadowed.size() == newSSTablesSize :
-                String.format("Expecting new size of %d, got %d while replacing %s by %s in %s",
-                          newSSTablesSize, newSSTables.size() + newShadowed.size(), oldSSTables, replacements, this);
-            newShadowed = ImmutableSet.copyOf(newShadowed);
-            newSSTables = ImmutableMap.copyOf(newSSTables);
-            SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables.keySet());
-            return new View(liveMemtables, flushingMemtables, newSSTables, compacting, newShadowed, intervalTree);
-        }
-
-        View markCompacting(Iterable<SSTableReader> tomark)
-        {
-            Set<SSTableReader> compactingNew = ImmutableSet.<SSTableReader>builder().addAll(compacting).addAll(tomark).build();
-            return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
-        }
-
-        View unmarkCompacting(Iterable<SSTableReader> tounmark)
-        {
-            Set<SSTableReader> compactingNew = ImmutableSet.copyOf(Sets.difference(compacting, ImmutableSet.copyOf(tounmark)));
-            return new View(liveMemtables, flushingMemtables, sstablesMap, compactingNew, shadowed, intervalTree);
-        }
-
-        @Override
-        public String toString()
-        {
-            return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
-        }
-
-        public List<SSTableReader> sstablesInBounds(AbstractBounds<RowPosition> rowBounds)
-        {
-            if (intervalTree.isEmpty())
-                return Collections.emptyList();
-            RowPosition stopInTree = rowBounds.right.isMinimum() ? intervalTree.max() : rowBounds.right;
-            return intervalTree.search(Interval.<RowPosition, SSTableReader>create(rowBounds.left, stopInTree));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 589958e..df8820b 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -250,7 +250,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
     protected synchronized void compact()
     {
         ArrayList<Descriptor> descriptors = new ArrayList<>();
-        for (SSTable sstable : hintStore.getDataTracker().getUncompactingSSTables())
+        for (SSTable sstable : hintStore.getTracker().getUncompacting())
             descriptors.add(sstable.descriptor);
 
         if (descriptors.isEmpty())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index f30bdaa..1d86784 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
 
 import com.google.common.base.Function;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -147,7 +146,7 @@ public class Keyspace
     }
 
     /**
-     * Removes every SSTable in the directory from the appropriate DataTracker's view.
+     * Removes every SSTable in the directory from the appropriate Tracker's view.
      * @param directory the unreadable directory, possibly with SSTables in it, but not necessarily.
      */
     public static void removeUnreadableSSTables(File directory)
@@ -276,6 +275,18 @@ public class Keyspace
         }
     }
 
+    private Keyspace(KSMetaData metadata)
+    {
+        this.metadata = metadata;
+        createReplicationStrategy(metadata);
+        this.metric = new KeyspaceMetrics(this);
+    }
+
+    public static Keyspace mockKS(KSMetaData metadata)
+    {
+        return new Keyspace(metadata);
+    }
+
     public void createReplicationStrategy(KSMetaData ksm)
     {
         replicationStrategy = AbstractReplicationStrategy.createReplicationStrategy(ksm.name,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index eab64ae..55b0bfe 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -27,6 +27,9 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
@@ -46,7 +49,7 @@ import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.*;
 
-public class Memtable
+public class Memtable implements Comparable<Memtable>
 {
     private static final Logger logger = LoggerFactory.getLogger(Memtable.class);
 
@@ -64,6 +67,11 @@ public class Memtable
     // the "first" ReplayPosition owned by this Memtable; this is inaccurate, and only used as a convenience to prevent CLSM flushing wantonly
     private final ReplayPosition minReplayPosition = CommitLog.instance.getContext();
 
+    public int compareTo(Memtable that)
+    {
+        return this.minReplayPosition.compareTo(that.minReplayPosition);
+    }
+
     public static final class LastReplayPosition extends ReplayPosition
     {
         public LastReplayPosition(ReplayPosition copy) {
@@ -92,6 +100,15 @@ public class Memtable
         this.cfs.scheduleFlush();
     }
 
+    // ONLY to be used for testing, to create a mock Memtable
+    @VisibleForTesting
+    public Memtable(CFMetaData metadata)
+    {
+        this.initialComparator = metadata.comparator;
+        this.cfs = null;
+        this.allocator = null;
+    }
+
     public MemtableAllocator getAllocator()
     {
         return allocator;
@@ -107,7 +124,8 @@ public class Memtable
         return currentOperations.get();
     }
 
-    void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
+    @VisibleForTesting
+    public void setDiscarding(OpOrder.Barrier writeBarrier, AtomicReference<ReplayPosition> lastReplayPosition)
     {
         assert this.writeBarrier == null;
         this.lastReplayPosition = lastReplayPosition;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 29826b8..38107c0 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -30,6 +30,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Memtable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -175,9 +176,9 @@ public abstract class AbstractCompactionStrategy
      */
     public abstract AbstractCompactionTask getUserDefinedTask(Collection<SSTableReader> sstables, final int gcBefore);
 
-    public AbstractCompactionTask getCompactionTask(Collection<SSTableReader> sstables, final int gcBefore, long maxSSTableBytes)
+    public AbstractCompactionTask getCompactionTask(LifecycleTransaction txn, final int gcBefore, long maxSSTableBytes)
     {
-        return new CompactionTask(cfs, sstables, gcBefore, false);
+        return new CompactionTask(cfs, txn, gcBefore, false);
     }
 
     /**
@@ -231,7 +232,7 @@ public abstract class AbstractCompactionStrategy
      */
     public void replaceFlushed(Memtable memtable, SSTableReader sstable)
     {
-        cfs.getDataTracker().replaceFlushed(memtable, sstable);
+        cfs.getTracker().replaceFlushed(memtable, sstable);
         if (sstable != null)
             CompactionManager.instance.submitBackground(cfs);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
index ac646ef..3bf224e 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionTask.java
@@ -24,27 +24,28 @@ import org.apache.cassandra.db.compaction.CompactionManager.CompactionExecutorSt
 import org.apache.cassandra.db.compaction.writers.CompactionAwareWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 
 public abstract class AbstractCompactionTask extends WrappedRunnable
 {
     protected final ColumnFamilyStore cfs;
-    protected Set<SSTableReader> sstables;
+    protected LifecycleTransaction transaction;
     protected boolean isUserDefined;
     protected OperationType compactionType;
 
     /**
      * @param cfs
-     * @param sstables must be marked compacting
+     * @param transaction the modifying managing the status of the sstables we're replacing
      */
-    public AbstractCompactionTask(ColumnFamilyStore cfs, Set<SSTableReader> sstables)
+    public AbstractCompactionTask(ColumnFamilyStore cfs, LifecycleTransaction transaction)
     {
         this.cfs = cfs;
-        this.sstables = sstables;
+        this.transaction = transaction;
         this.isUserDefined = false;
         this.compactionType = OperationType.COMPACTION;
         // enforce contract that caller should mark sstables compacting
-        Set<SSTableReader> compacting = cfs.getDataTracker().getCompacting();
-        for (SSTableReader sstable : sstables)
+        Set<SSTableReader> compacting = transaction.tracker.getCompacting();
+        for (SSTableReader sstable : transaction.originals())
             assert compacting.contains(sstable) : sstable.getFilename() + " is not correctly marked compacting";
     }
 
@@ -59,10 +60,10 @@ public abstract class AbstractCompactionTask extends WrappedRunnable
         }
         finally
         {
-            cfs.getDataTracker().unmarkCompacting(sstables);
+            transaction.close();
         }
     }
-    public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, Set<SSTableReader> allSSTables, Set<SSTableReader> nonExpiredSSTables);
+    public abstract CompactionAwareWriter getCompactionAwareWriter(ColumnFamilyStore cfs, LifecycleTransaction txn, Set<SSTableReader> nonExpiredSSTables);
 
     protected abstract int executeInternal(CompactionExecutorStatsCollector collector);
 
@@ -80,6 +81,6 @@ public abstract class AbstractCompactionTask extends WrappedRunnable
 
     public String toString()
     {
-        return "CompactionTask(" + sstables + ")";
+        return "CompactionTask(" + transaction + ")";
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e5a76bdb/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index a49a3ea..2292e01 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -24,6 +24,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.lifecycle.SSTableIntervalTree;
+import org.apache.cassandra.db.lifecycle.Tracker;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowPosition;
 import org.apache.cassandra.utils.AlwaysPresentFilter;
@@ -31,7 +33,7 @@ import org.apache.cassandra.utils.AlwaysPresentFilter;
 import org.apache.cassandra.utils.OverlapIterator;
 import org.apache.cassandra.utils.concurrent.Refs;
 
-import static org.apache.cassandra.db.DataTracker.buildIntervals;
+import static org.apache.cassandra.db.lifecycle.SSTableIntervalTree.buildIntervals;
 
 /**
  * Manage compaction options.