You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/08/05 13:48:52 UTC

[01/23] cassandra git commit: Disable passing control to post-flush after flush failure to prevent data loss.

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 bc0d1da0c -> bd6654733
  refs/heads/cassandra-2.2 039852126 -> 6dc1745ed
  refs/heads/cassandra-3.0 b66e5a189 -> 904cb5d10
  refs/heads/cassandra-3.9 21c92cab8 -> 7b1021733
  refs/heads/trunk 7fe430943 -> 624ed7838


Disable passing control to post-flush after flush failure to prevent
data loss.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828

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

Branch: refs/heads/cassandra-2.1
Commit: bd6654733dded3513c2c7acf96df2c364b0c043e
Parents: bc0d1da
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 3 11:32:48 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:35:25 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  45 ++++--
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  87 ++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 311 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8ecc787..1275631 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/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 b64d5de..6e82745 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -99,6 +99,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                               new NamedThreadFactory("MemtablePostFlush"),
                                                                                               "internal");
 
+    // If a flush fails with an error the post-flush is never allowed to continue. This stores the error that caused it
+    // to be able to show an error on following flushes instead of blindly continuing.
+    private static volatile FSWriteError previousFlushFailure = null;
+
     private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
                                                                                             StageManager.KEEPALIVE,
                                                                                             TimeUnit.SECONDS,
@@ -869,12 +873,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         synchronized (data)
         {
+            if (previousFlushFailure != null)
+                throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                              + "no flushes can be carried out until the node is restarted.",
+                                                previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            flushExecutor.execute(flush);
+            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            flushExecutor.submit(flushTask);
             ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
             postFlushExecutor.submit(task);
-            return task;
+
+            @SuppressWarnings("unchecked")
+            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            return future;
         }
     }
 
@@ -967,7 +979,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         final ReplayPosition lastReplayPosition;
-        volatile FSWriteError flushFailure = null;
 
         private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
         {
@@ -1010,16 +1021,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             // must check lastReplayPosition != null because Flush may find that all memtables are clean
             // and so not set a lastReplayPosition
-            // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (lastReplayPosition != null && flushFailure == null)
+            if (lastReplayPosition != null)
             {
                 CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
             }
 
             metric.pendingFlushes.dec();
-
-            if (flushFailure != null)
-                throw flushFailure;
         }
     }
 
@@ -1127,16 +1134,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
                     reclaim(memtable);
                 }
+
+                // signal the post-flush we've done our work
+                // Note: This should not be done in case of error. Read more below.
+                postFlush.latch.countDown();
             }
             catch (FSWriteError e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
-                postFlush.flushFailure = e;
+                // The call above may kill the process or the transports, or ignore the error.
+                // In any case we should not be passing on control to post-flush as a subsequent succeeding flush
+                // could mask the error and:
+                //   - let the commit log discard unpersisted data, resulting in data loss
+                //   - let truncations proceed, with the possibility of resurrecting the unflushed data
+                //   - let snapshots succeed with incomplete data
+
+                // Not passing control on means that all flushes from the moment of failure cannot complete
+                // (including snapshots).
+                // If the disk failure policy is ignore, this will cause memtables and the commit log to grow
+                // unboundedly until the node eventually fails.
+                previousFlushFailure = e;
+                throw e;
             }
-
-            // signal the post-flush we've done our work
-            postFlush.latch.countDown();
         }
 
         private void reclaim(final Memtable memtable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
new file mode 100644
index 0000000..c0023dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOError;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories.DataDirectory;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.CommitLogSegment;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSWriteError;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+ */
+public class OutOfSpaceBase extends CQLTester
+{
+    public void makeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+    }
+
+    public void markDirectoriesUnwriteable()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+    }
+
+    public void flushAndExpectError() throws InterruptedException, ExecutionException
+    {
+        try
+        {
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+            fail("FSWriteError expected.");
+        }
+        catch (ExecutionException e)
+        {
+            // Correct path.
+            Assert.assertTrue(e.getCause() instanceof FSWriteError);
+        }
+
+        // Make sure commit log wasn't discarded.
+        UUID cfid = currentTableMetadata().cfId;
+        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+            if (segment.getDirtyCFIDs().contains(cfid))
+                return;
+        fail("Expected commit log to remain dirty for the affected table.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
new file mode 100644
index 0000000..46d71e4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceDieTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableDie() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
+            flushAndExpectError();
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
new file mode 100644
index 0000000..854de80
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceIgnoreTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableIgnore() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+            flushAndExpectError();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
new file mode 100644
index 0000000..b48df56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceStopTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableStop() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
+            flushAndExpectError();
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
deleted file mode 100644
index 8304aff..0000000
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ /dev/null
@@ -1,157 +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.cql3;
-
-import static junit.framework.Assert.fail;
-
-import java.io.IOError;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config.DiskFailurePolicy;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class OutOfSpaceTest extends CQLTester
-{
-    @Test
-    public void testFlushUnwriteableDie() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
-            flushAndExpectError();
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableStop() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
-            flushAndExpectError();
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableIgnore() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
-            flushAndExpectError();
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-
-        // Next flush should succeed.
-        makeTable();
-        flush();
-    }
-
-    public void makeTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-    }
-
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.directories.getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
-    public void flushAndExpectError() throws InterruptedException, ExecutionException
-    {
-        try
-        {
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
-            fail("FSWriteError expected.");
-        }
-        catch (ExecutionException e)
-        {
-            // Correct path.
-            Assert.assertTrue(e.getCause() instanceof FSWriteError);
-        }
-
-        // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
-        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
-                return;
-        fail("Expected commit log to remain dirty for the affected table.");
-    }
-}


[18/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
Change commitlog and sstables to track dirty and clean intervals.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828


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

Branch: refs/heads/cassandra-3.9
Commit: 904cb5d10e0de1a6ca89249be8c257ed38a80ef0
Parents: cf85f52
Author: Branimir Lambov <br...@datastax.com>
Authored: Sat May 14 11:31:16 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:38:37 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  66 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  18 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         |  59 +++-
 .../db/commitlog/CommitLogSegment.java          |  77 ++---
 .../db/commitlog/CommitLogSegmentManager.java   |   4 +-
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../cassandra/db/commitlog/ReplayPosition.java  |  71 ----
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  44 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  36 +-
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  14 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  38 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   2 +-
 test/unit/org/apache/cassandra/Util.java        |  21 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 159 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 98 files changed, 1229 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 046c8b3..b596fc9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
  * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
  * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
  * Fix upgrade of super columns on thrift (CASSANDRA-12335)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
index f47fd57..bc733d7 100644
--- a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
+++ b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
@@ -29,6 +29,8 @@ import java.util.concurrent.CopyOnWriteArraySet;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
@@ -101,6 +103,17 @@ public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
     }
 
     /**
+     * Testing only!
+     * Clear the set of unwritable directories.
+     */
+    @VisibleForTesting
+    public static void clearUnwritableUnsafe()
+    {
+        instance.unwritableDirectories.clear();
+    }
+
+
+    /**
      * Tells whether or not the directory is blacklisted for reads.
      * @return whether or not the directory is blacklisted for reads.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 400fd36..82604e2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -179,9 +179,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    public static volatile ColumnFamilyStore discardFlushResults;
-
     public final Keyspace keyspace;
     public final String name;
     public final CFMetaData metadata;
@@ -926,25 +923,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         volatile FSWriteError flushFailure = null;
-        final ReplayPosition commitLogUpperBound;
         final List<Memtable> memtables;
-        final List<Collection<SSTableReader>> readers;
 
-        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
-                          List<Memtable> memtables, List<Collection<SSTableReader>> readers)
+        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier,
+                          List<Memtable> memtables)
         {
             this.writeBarrier = writeBarrier;
             this.flushSecondaryIndexes = flushSecondaryIndexes;
-            this.commitLogUpperBound = commitLogUpperBound;
             this.memtables = memtables;
-            this.readers = readers;
         }
 
         public ReplayPosition call()
         {
-            if (discardFlushResults == ColumnFamilyStore.this)
-                return commitLogUpperBound;
-
             writeBarrier.await();
 
             /**
@@ -968,17 +958,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 throw new IllegalStateException();
             }
 
+            ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (flushFailure == null)
+            if (flushFailure == null && !memtables.isEmpty())
             {
-                CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                for (int i = 0 ; i < memtables.size() ; i++)
-                {
-                    Memtable memtable = memtables.get(i);
-                    Collection<SSTableReader> reader = readers.get(i);
-                    memtable.cfs.data.permitCompactionOfFlushed(reader);
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, reader);
-                }
+                Memtable memtable = memtables.get(0);
+                commitLogUpperBound = memtable.getCommitLogUpperBound();
+                CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
             }
 
             metric.pendingFlushes.dec();
@@ -1002,7 +988,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         final OpOrder.Barrier writeBarrier;
         final List<Memtable> memtables = new ArrayList<>();
-        final List<Collection<SSTableReader>> readers = new ArrayList<>();
         final PostFlush postFlush;
         final boolean truncate;
 
@@ -1044,7 +1029,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
             // replay positions have also completed, i.e. the memtables are done and ready to flush
             writeBarrier.issue();
-            postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
+            postFlush = new PostFlush(!truncate, writeBarrier, memtables);
         }
 
         public void run()
@@ -1063,8 +1048,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 memtable.cfs.data.markFlushing(memtable);
                 if (memtable.isClean() || truncate)
                 {
-                    memtable.cfs.data.replaceFlushed(memtable, Collections.emptyList());
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, Collections.emptyList());
+                    memtable.cfs.replaceFlushed(memtable, Collections.emptyList());
                     reclaim(memtable);
                     iter.remove();
                 }
@@ -1077,9 +1061,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (Memtable memtable : memtables)
                 {
                     Collection<SSTableReader> readers = memtable.flush();
-                    memtable.cfs.data.replaceFlushed(memtable, readers);
+                    memtable.cfs.replaceFlushed(memtable, readers);
                     reclaim(memtable);
-                    this.readers.add(readers);
                 }
             }
             catch (FSWriteError e)
@@ -1126,21 +1109,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    // this method should ONLY be used for testing commit log behaviour; it discards the current memtable
-    // contents without marking the commit log clean, and prevents any proceeding flushes from marking
-    // the commit log as done, however they *will* terminate (unlike under typical failures) to ensure progress is made
-    public void simulateFailedFlush()
-    {
-        discardFlushResults = this;
-        data.markFlushing(data.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this)));
-    }
-
-    public void resumeFlushing()
-    {
-        discardFlushResults = null;
-    }
-
     /**
      * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
      * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
@@ -1483,16 +1451,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return data;
     }
 
-    public Collection<SSTableReader> getSSTables()
-    {
-        return data.getSSTables();
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompactSSTables()
-    {
-        return data.getPermittedToCompact();
-    }
-
     public Set<SSTableReader> getLiveSSTables()
     {
         return data.getView().liveSSTables();
@@ -2032,7 +1990,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         long now = System.currentTimeMillis();
         // make sure none of our sstables are somehow in the future (clock drift, perhaps)
         for (ColumnFamilyStore cfs : concatWithIndexes())
-            for (SSTableReader sstable : cfs.data.getSSTables())
+            for (SSTableReader sstable : cfs.getLiveSSTables())
                 now = Math.max(now, sstable.maxDataAge);
         truncatedAt = now;
 
@@ -2130,7 +2088,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             public LifecycleTransaction call() throws Exception
             {
                 assert data.getCompacting().isEmpty() : data.getCompacting();
-                Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                Iterable<SSTableReader> sstables = getLiveSSTables();
                 sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
                 sstables = ImmutableList.copyOf(sstables);
                 LifecycleTransaction modifier = data.tryModify(sstables, operationType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 01ffd52..877f984 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -385,7 +385,7 @@ public class Directories
 
         if (candidates.isEmpty())
             if (tooBig)
-                throw new RuntimeException("Insufficient disk space to write " + writeSize + " bytes");
+                throw new FSWriteError(new IOException("Insufficient disk space to write " + writeSize + " bytes"), "");
             else
                 throw new FSWriteError(new IOException("All configured data directories have been blacklisted as unwritable for erroring out"), "");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 93dc5af..3c77092 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
@@ -193,6 +194,11 @@ public class Memtable implements Comparable<Memtable>
         return commitLogLowerBound.get();
     }
 
+    public ReplayPosition getCommitLogUpperBound()
+    {
+        return commitLogUpperBound.get();
+    }
+
     public boolean isLive()
     {
         return allocator.isLive();
@@ -331,6 +337,15 @@ public class Memtable implements Comparable<Memtable>
         return minTimestamp;
     }
 
+    /**
+     * For testing only. Give this memtable too big a size to make it always fail flushing.
+     */
+    @VisibleForTesting
+    public void makeUnflushable()
+    {
+        liveDataSize.addAndGet(1L * 1024 * 1024 * 1024 * 1024 * 1024);
+    }
+
     private long estimatedSize()
     {
         long keySize = 0;
@@ -418,8 +433,7 @@ public class Memtable implements Comparable<Memtable>
         {
             txn = LifecycleTransaction.offline(OperationType.FLUSH);
             MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
-                                                         .commitLogLowerBound(commitLogLowerBound.get())
-                                                         .commitLogUpperBound(commitLogUpperBound.get());
+                    .commitLogIntervals(new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));
 
             return new SSTableTxnWriter(txn,
                                         cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index dcdd855..dfe3f91 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -290,11 +290,12 @@ public class CommitLog implements CommitLogMBean
      * given. Discards any commit log segments that are no longer used.
      *
      * @param cfId    the column family ID that was flushed
-     * @param context the replay position of the flush
+     * @param lowerBound the lowest covered replay position of the flush
+     * @param lowerBound the highest covered replay position of the flush
      */
-    public void discardCompletedSegments(final UUID cfId, final ReplayPosition context)
+    public void discardCompletedSegments(final UUID cfId, final ReplayPosition lowerBound, final ReplayPosition upperBound)
     {
-        logger.trace("discard completed log segments for {}, table {}", context, cfId);
+        logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
@@ -303,7 +304,7 @@ public class CommitLog implements CommitLogMBean
         for (Iterator<CommitLogSegment> iter = allocator.getActiveSegments().iterator(); iter.hasNext();)
         {
             CommitLogSegment segment = iter.next();
-            segment.markClean(cfId, context);
+            segment.markClean(cfId, lowerBound, upperBound);
 
             if (segment.isUnused())
             {
@@ -318,7 +319,7 @@ public class CommitLog implements CommitLogMBean
 
             // Don't mark or try to delete any newer segments once we've reached the one containing the
             // position of the flush.
-            if (segment.contains(context))
+            if (segment.contains(upperBound))
                 break;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index f45a47a..af8efb4 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -35,6 +35,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.commons.lang3.StringUtils;
@@ -52,6 +53,7 @@ import org.apache.cassandra.io.util.FileSegmentInputStream;
 import org.apache.cassandra.io.util.RebufferingInputStream;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.FileDataInput;
@@ -74,7 +76,7 @@ public class CommitLogReplayer
     private final List<Future<?>> futures;
     private final Map<UUID, AtomicInteger> invalidMutations;
     private final AtomicInteger replayedCount;
-    private final Map<UUID, ReplayPosition.ReplayFilter> cfPersisted;
+    private final Map<UUID, IntervalSet<ReplayPosition>> cfPersisted;
     private final ReplayPosition globalPosition;
     private final CRC32 checksum;
     private byte[] buffer;
@@ -83,7 +85,7 @@ public class CommitLogReplayer
     private final ReplayFilter replayFilter;
     private final CommitLogArchiver archiver;
 
-    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, ReplayPosition.ReplayFilter> cfPersisted, ReplayFilter replayFilter)
+    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, IntervalSet<ReplayPosition>> cfPersisted, ReplayFilter replayFilter)
     {
         this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
@@ -101,10 +103,9 @@ public class CommitLogReplayer
 
     public static CommitLogReplayer construct(CommitLog commitLog)
     {
-        // compute per-CF and global replay positions
-        Map<UUID, ReplayPosition.ReplayFilter> cfPersisted = new HashMap<>();
+        // compute per-CF and global replay intervals
+        Map<UUID, IntervalSet<ReplayPosition>> cfPersisted = new HashMap<>();
         ReplayFilter replayFilter = ReplayFilter.create();
-        ReplayPosition globalPosition = null;
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
@@ -129,14 +130,10 @@ public class CommitLogReplayer
                 }
             }
 
-            ReplayPosition.ReplayFilter filter = new ReplayPosition.ReplayFilter(cfs.getSSTables(), truncatedAt);
-            if (!filter.isEmpty())
-                cfPersisted.put(cfs.metadata.cfId, filter);
-            else
-                globalPosition = ReplayPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
+            IntervalSet<ReplayPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
+            cfPersisted.put(cfs.metadata.cfId, filter);
         }
-        if (globalPosition == null)
-            globalPosition = ReplayPosition.firstNotCovered(cfPersisted.values());
+        ReplayPosition globalPosition = firstNotCovered(cfPersisted.values());
         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
         return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
     }
@@ -148,6 +145,41 @@ public class CommitLogReplayer
             recover(clogs[i], i + 1 == clogs.length);
     }
 
+    /**
+     * A set of known safe-to-discard commit log replay positions, based on
+     * the range covered by on disk sstables and those prior to the most recent truncation record
+     */
+    public static IntervalSet<ReplayPosition> persistedIntervals(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
+    {
+        IntervalSet.Builder<ReplayPosition> builder = new IntervalSet.Builder<>();
+        for (SSTableReader reader : onDisk)
+            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+
+        if (truncatedAt != null)
+            builder.add(ReplayPosition.NONE, truncatedAt);
+        return builder.build();
+    }
+
+    /**
+     * Find the earliest commit log position that is not covered by the known flushed ranges for some table.
+     *
+     * For efficiency this assumes that the first contiguously flushed interval we know of contains the moment that the
+     * given table was constructed* and hence we can start replay from the end of that interval.
+     *
+     * If such an interval is not known, we must replay from the beginning.
+     *
+     * * This is not true only until if the very first flush of a table stalled or failed, while the second or latter
+     *   succeeded. The chances of this happening are at most very low, and if the assumption does prove to be
+     *   incorrect during replay there is little chance that the affected deployment is in production.
+     */
+    public static ReplayPosition firstNotCovered(Collection<IntervalSet<ReplayPosition>> ranges)
+    {
+        return ranges.stream()
+                .map(intervals -> Iterables.getFirst(intervals.ends(), ReplayPosition.NONE)) 
+                .min(Ordering.natural())
+                .get(); // iteration is per known-CF, there must be at least one. 
+    }
+
     public int blockForWrites()
     {
         for (Map.Entry<UUID, AtomicInteger> entry : invalidMutations.entrySet())
@@ -293,8 +325,7 @@ public class CommitLogReplayer
      */
     private boolean shouldReplay(UUID cfId, ReplayPosition position)
     {
-        ReplayPosition.ReplayFilter filter = cfPersisted.get(cfId);
-        return filter == null || filter.shouldReplay(position);
+        return !cfPersisted.get(cfId).contains(position);
     }
 
     @SuppressWarnings("resource")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 27c05b4..d2f12bf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -101,11 +102,11 @@ public abstract class CommitLogSegment
     // a signal for writers to wait on to confirm the log message they provided has been written to disk
     private final WaitQueue syncComplete = new WaitQueue();
 
-    // a map of Cf->dirty position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final NonBlockingHashMap<UUID, AtomicInteger> cfDirty = new NonBlockingHashMap<>(1024);
+    // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
+    private final NonBlockingHashMap<UUID, IntegerInterval> cfDirty = new NonBlockingHashMap<>(1024);
 
-    // a map of Cf->clean position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final ConcurrentHashMap<UUID, AtomicInteger> cfClean = new ConcurrentHashMap<>();
+    // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use
+    private final ConcurrentHashMap<UUID, IntegerInterval.Set> cfClean = new ConcurrentHashMap<>();
 
     public final long id;
 
@@ -423,10 +424,23 @@ public abstract class CommitLogSegment
         }
     }
 
+    public static<K> void coverInMap(ConcurrentMap<K, IntegerInterval> map, K key, int value)
+    {
+        IntegerInterval i = map.get(key);
+        if (i == null)
+        {
+            i = map.putIfAbsent(key, new IntegerInterval(value, value));
+            if (i == null)
+                // success
+                return;
+        }
+        i.expandToCover(value);
+    }
+
     void markDirty(Mutation mutation, int allocatedPosition)
     {
         for (PartitionUpdate update : mutation.getPartitionUpdates())
-            ensureAtleast(cfDirty, update.metadata().cfId, allocatedPosition);
+            coverInMap(cfDirty, update.metadata().cfId, allocatedPosition);
     }
 
     /**
@@ -437,55 +451,32 @@ public abstract class CommitLogSegment
      * @param cfId    the column family ID that is now clean
      * @param context the optional clean offset
      */
-    public synchronized void markClean(UUID cfId, ReplayPosition context)
+    public synchronized void markClean(UUID cfId, ReplayPosition startPosition, ReplayPosition endPosition)
     {
+        if (startPosition.segment > id || endPosition.segment < id)
+            return;
         if (!cfDirty.containsKey(cfId))
             return;
-        if (context.segment == id)
-            markClean(cfId, context.position);
-        else if (context.segment > id)
-            markClean(cfId, Integer.MAX_VALUE);
-    }
-
-    private void markClean(UUID cfId, int position)
-    {
-        ensureAtleast(cfClean, cfId, position);
+        int start = startPosition.segment == id ? startPosition.position : 0;
+        int end = endPosition.segment == id ? endPosition.position : Integer.MAX_VALUE;
+        cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
         removeCleanFromDirty();
     }
 
-    private static void ensureAtleast(ConcurrentMap<UUID, AtomicInteger> map, UUID cfId, int value)
-    {
-        AtomicInteger i = map.get(cfId);
-        if (i == null)
-        {
-            AtomicInteger i2 = map.putIfAbsent(cfId, i = new AtomicInteger());
-            if (i2 != null)
-                i = i2;
-        }
-        while (true)
-        {
-            int cur = i.get();
-            if (cur > value)
-                break;
-            if (i.compareAndSet(cur, value))
-                break;
-        }
-    }
-
     private void removeCleanFromDirty()
     {
         // if we're still allocating from this segment, don't touch anything since it can't be done thread-safely
         if (isStillAllocating())
             return;
 
-        Iterator<Map.Entry<UUID, AtomicInteger>> iter = cfClean.entrySet().iterator();
+        Iterator<Map.Entry<UUID, IntegerInterval.Set>> iter = cfClean.entrySet().iterator();
         while (iter.hasNext())
         {
-            Map.Entry<UUID, AtomicInteger> clean = iter.next();
+            Map.Entry<UUID, IntegerInterval.Set> clean = iter.next();
             UUID cfId = clean.getKey();
-            AtomicInteger cleanPos = clean.getValue();
-            AtomicInteger dirtyPos = cfDirty.get(cfId);
-            if (dirtyPos != null && dirtyPos.intValue() <= cleanPos.intValue())
+            IntegerInterval.Set cleanSet = clean.getValue();
+            IntegerInterval dirtyInterval = cfDirty.get(cfId);
+            if (dirtyInterval != null && cleanSet.covers(dirtyInterval))
             {
                 cfDirty.remove(cfId);
                 iter.remove();
@@ -502,12 +493,12 @@ public abstract class CommitLogSegment
             return cfDirty.keySet();
 
         List<UUID> r = new ArrayList<>(cfDirty.size());
-        for (Map.Entry<UUID, AtomicInteger> dirty : cfDirty.entrySet())
+        for (Map.Entry<UUID, IntegerInterval> dirty : cfDirty.entrySet())
         {
             UUID cfId = dirty.getKey();
-            AtomicInteger dirtyPos = dirty.getValue();
-            AtomicInteger cleanPos = cfClean.get(cfId);
-            if (cleanPos == null || cleanPos.intValue() < dirtyPos.intValue())
+            IntegerInterval dirtyInterval = dirty.getValue();
+            IntegerInterval.Set cleanSet = cfClean.get(cfId);
+            if (cleanSet == null || !cleanSet.covers(dirtyInterval))
                 r.add(dirty.getKey());
         }
         return r;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 66ad6a3..82cee50 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -310,7 +310,7 @@ public class CommitLogSegmentManager
 
             for (CommitLogSegment segment : activeSegments)
                 for (UUID cfId : droppedCfs)
-                    segment.markClean(cfId, segment.getContext());
+                    segment.markClean(cfId, ReplayPosition.NONE, segment.getContext());
 
             // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
             // if the previous active segment was the only one to recycle (since an active segment isn't
@@ -451,7 +451,7 @@ public class CommitLogSegmentManager
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
                     logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                    segment.markClean(dirtyCFId, segment.getContext());
+                    segment.markClean(dirtyCFId, ReplayPosition.NONE, segment.getContext());
                 }
                 else if (!flushes.containsKey(dirtyCFId))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
new file mode 100644
index 0000000..bd0ea22
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
@@ -0,0 +1,192 @@
+package org.apache.cassandra.db.commitlog;
+
+import java.io.IOException;
+import java.util.*;
+
+import com.google.common.collect.ImmutableSortedMap;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An immutable set of closed intervals, stored in normalized form (i.e. where overlapping intervals are converted
+ * to a single interval covering both).
+ *
+ * The set is stored as a sorted map from interval starts to the corresponding end. The map satisfies
+ *   curr().getKey() <= curr().getValue() < next().getKey()
+ */
+public class IntervalSet<T extends Comparable<T>>
+{
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static final IntervalSet EMPTY = new IntervalSet(ImmutableSortedMap.of());
+
+    final private NavigableMap<T, T> ranges;
+
+    private IntervalSet(ImmutableSortedMap<T, T> ranges)
+    {
+        this.ranges = ranges;
+    }
+
+    /**
+     * Construct new set containing the interval with the given start and end position.
+     */
+    public IntervalSet(T start, T end)
+    {
+        this(ImmutableSortedMap.of(start, end));
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T extends Comparable<T>> IntervalSet<T> empty()
+    {
+        return (IntervalSet<T>) EMPTY;
+    }
+
+    public boolean contains(T position)
+    {
+        // closed (i.e. inclusive) intervals
+        Map.Entry<T, T> range = ranges.floorEntry(position);
+        return range != null && position.compareTo(range.getValue()) <= 0;
+    }
+
+    public boolean isEmpty()
+    {
+        return ranges.isEmpty();
+    }
+
+    public Optional<T> lowerBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.firstKey());
+    }
+
+    public Optional<T> upperBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.lastEntry().getValue());
+    }
+
+    public Collection<T> starts()
+    {
+        return ranges.keySet();
+    }
+
+    public Collection<T> ends()
+    {
+        return ranges.values();
+    }
+
+    public String toString()
+    {
+        return ranges.toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return ranges.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        return obj instanceof IntervalSet && ranges.equals(((IntervalSet<?>) obj).ranges);
+    }
+
+    public static final <T extends Comparable<T>> ISerializer<IntervalSet<T>> serializer(ISerializer<T> pointSerializer)
+    {
+        return new ISerializer<IntervalSet<T>>()
+        {
+            public void serialize(IntervalSet<T> intervals, DataOutputPlus out) throws IOException
+            {
+                out.writeInt(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    pointSerializer.serialize(en.getKey(), out);
+                    pointSerializer.serialize(en.getValue(), out);
+                }
+            }
+    
+            public IntervalSet<T> deserialize(DataInputPlus in) throws IOException
+            {
+                int count = in.readInt();
+                NavigableMap<T, T> ranges = new TreeMap<>();
+                for (int i = 0; i < count; ++i)
+                    ranges.put(pointSerializer.deserialize(in), pointSerializer.deserialize(in));
+                return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+            }
+    
+            public long serializedSize(IntervalSet<T> intervals)
+            {
+                long size = TypeSizes.sizeof(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    size += pointSerializer.serializedSize(en.getKey());
+                    size += pointSerializer.serializedSize(en.getValue());
+                }
+                return size;
+            }
+        };
+    };
+
+    /**
+     * Builder of interval sets, applying the necessary normalization while adding ranges.
+     *
+     * Data is stored as above, as a sorted map from interval starts to the corresponding end, which satisfies
+     *   curr().getKey() <= curr().getValue() < next().getKey()
+     */
+    static public class Builder<T extends Comparable<T>>
+    {
+        final NavigableMap<T, T> ranges;
+
+        public Builder()
+        {
+            this.ranges = new TreeMap<>();
+        }
+
+        public Builder(T start, T end)
+        {
+            this();
+            assert start.compareTo(end) <= 0;
+            ranges.put(start, end);
+        }
+
+        /**
+         * Add an interval to the set and perform normalization.
+         */
+        public void add(T start, T end)
+        {
+            assert start.compareTo(end) <= 0;
+            // extend ourselves to cover any ranges we overlap
+            // record directly preceding our end may extend past us, so take the max of our end and its
+            Map.Entry<T, T> extend = ranges.floorEntry(end);
+            if (extend != null && extend.getValue().compareTo(end) > 0)
+                end = extend.getValue();
+
+            // record directly preceding our start may extend into us; if it does, we take it as our start
+            extend = ranges.lowerEntry(start);
+            if (extend != null && extend.getValue().compareTo(start) >= 0)
+                start = extend.getKey();
+
+            // remove all covered intervals
+            // since we have adjusted start and end to cover the ones that would be only partially covered, we
+            // are certain that anything whose start falls within the span is completely covered
+            ranges.subMap(start, end).clear();
+            // add the new interval
+            ranges.put(start, end);
+        }
+
+        public void addAll(IntervalSet<T> otherSet)
+        {
+            for (Map.Entry<T, T> en : otherSet.ranges.entrySet())
+            {
+                add(en.getKey(), en.getValue());
+            }
+        }
+
+        public IntervalSet<T> build()
+        {
+            return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
index 0b21763..b0214b8 100644
--- a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
+++ b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
@@ -18,15 +18,9 @@
 package org.apache.cassandra.db.commitlog;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
@@ -43,71 +37,6 @@ public class ReplayPosition implements Comparable<ReplayPosition>
     public final long segment;
     public final int position;
 
-    /**
-     * A filter of known safe-to-discard commit log replay positions, based on
-     * the range covered by on disk sstables and those prior to the most recent truncation record
-     */
-    public static class ReplayFilter
-    {
-        final NavigableMap<ReplayPosition, ReplayPosition> persisted = new TreeMap<>();
-        public ReplayFilter(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
-        {
-            for (SSTableReader reader : onDisk)
-            {
-                ReplayPosition start = reader.getSSTableMetadata().commitLogLowerBound;
-                ReplayPosition end = reader.getSSTableMetadata().commitLogUpperBound;
-                add(persisted, start, end);
-            }
-            if (truncatedAt != null)
-                add(persisted, ReplayPosition.NONE, truncatedAt);
-        }
-
-        private static void add(NavigableMap<ReplayPosition, ReplayPosition> ranges, ReplayPosition start, ReplayPosition end)
-        {
-            // extend ourselves to cover any ranges we overlap
-            // record directly preceding our end may extend past us, so take the max of our end and its
-            Map.Entry<ReplayPosition, ReplayPosition> extend = ranges.floorEntry(end);
-            if (extend != null && extend.getValue().compareTo(end) > 0)
-                end = extend.getValue();
-
-            // record directly preceding our start may extend into us; if it does, we take it as our start
-            extend = ranges.lowerEntry(start);
-            if (extend != null && extend.getValue().compareTo(start) >= 0)
-                start = extend.getKey();
-
-            ranges.subMap(start, end).clear();
-            ranges.put(start, end);
-        }
-
-        public boolean shouldReplay(ReplayPosition position)
-        {
-            // replay ranges are start exclusive, end inclusive
-            Map.Entry<ReplayPosition, ReplayPosition> range = persisted.lowerEntry(position);
-            return range == null || position.compareTo(range.getValue()) > 0;
-        }
-
-        public boolean isEmpty()
-        {
-            return persisted.isEmpty();
-        }
-    }
-
-    public static ReplayPosition firstNotCovered(Iterable<ReplayFilter> ranges)
-    {
-        ReplayPosition min = null;
-        for (ReplayFilter map : ranges)
-        {
-            ReplayPosition first = map.persisted.firstEntry().getValue();
-            if (min == null)
-                min = first;
-            else
-                min = Ordering.natural().min(min, first);
-        }
-        if (min == null)
-            return NONE;
-        return min;
-    }
-
     public ReplayPosition(long segment, int position)
     {
         this.segment = segment;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 0dce52b..a80a6f4 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -241,6 +241,9 @@ public abstract class AbstractCompactionStrategy
      */
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 444d43d..a9bfbd2 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -189,6 +189,9 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     public int getUnleveledSSTables()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index c94b88f..5a3d524 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -37,7 +37,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
@@ -48,8 +47,6 @@ import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static com.google.common.base.Predicates.and;
-import static com.google.common.base.Predicates.in;
-import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
@@ -204,7 +201,6 @@ public class Tracker
                          ImmutableList.<Memtable>of(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
                          SSTableIntervalTree.empty()));
     }
 
@@ -351,49 +347,19 @@ public class Tracker
 
         Throwable fail;
         fail = updateSizeTracking(emptySet(), sstables, null);
+        // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+        fail = notifyAdded(sstables, fail);
+
+        if (!isDummy() && !cfstore.isValid())
+            dropSSTables();
 
         maybeFail(fail);
     }
 
-    /**
-     * permit compaction of the provided sstable; this translates to notifying compaction
-     * strategies of its existence, and potentially submitting a background task
-     */
-    public void permitCompactionOfFlushed(Collection<SSTableReader> sstables)
-    {
-        if (sstables.isEmpty())
-            return;
-
-        apply(View.permitCompactionOfFlushed(sstables));
-
-        if (isDummy())
-            return;
-
-        if (cfstore.isValid())
-        {
-            notifyAdded(sstables);
-            CompactionManager.instance.submitBackground(cfstore);
-        }
-        else
-        {
-            dropSSTables();
-        }
-    }
 
 
     // MISCELLANEOUS public utility calls
 
-    public Set<SSTableReader> getSSTables()
-    {
-        return view.get().sstables;
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompact()
-    {
-        View view = this.view.get();
-        return filter(view.sstables, not(in(view.premature)));
-    }
-
     public Set<SSTableReader> getCompacting()
     {
         return view.get().compacting;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index 3fa197f..4b3aae0 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -33,7 +33,6 @@ 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.in;
 import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableList.copyOf;
 import static com.google.common.collect.ImmutableList.of;
@@ -70,7 +69,6 @@ public class View
     public final List<Memtable> flushingMemtables;
     final Set<SSTableReader> compacting;
     final Set<SSTableReader> sstables;
-    final Set<SSTableReader> premature;
     // 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
@@ -80,7 +78,7 @@ public class View
 
     final SSTableIntervalTree intervalTree;
 
-    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, Set<SSTableReader> premature, SSTableIntervalTree intervalTree)
+    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, SSTableIntervalTree intervalTree)
     {
         assert liveMemtables != null;
         assert flushingMemtables != null;
@@ -95,7 +93,6 @@ public class View
         this.sstables = sstablesMap.keySet();
         this.compactingMap = compacting;
         this.compacting = compactingMap.keySet();
-        this.premature = premature;
         this.intervalTree = intervalTree;
     }
 
@@ -256,7 +253,7 @@ public class View
                 assert all(mark, Helpers.idIn(view.sstablesMap));
                 return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
                                 replace(view.compactingMap, unmark, mark),
-                                view.premature, view.intervalTree);
+                                view.intervalTree);
             }
         };
     }
@@ -270,7 +267,7 @@ public class View
             public boolean apply(View view)
             {
                 for (SSTableReader reader : readers)
-                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted() || view.premature.contains(reader))
+                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted())
                         return false;
                 return true;
             }
@@ -287,7 +284,7 @@ public class View
             public View apply(View view)
             {
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
-                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap, view.premature,
+                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
@@ -302,7 +299,7 @@ public class 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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -321,7 +318,7 @@ public class View
                                                            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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, newFlushing, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -338,32 +335,15 @@ public class View
 
                 if (flushed == null || flushed.isEmpty())
                     return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
-                                    view.compactingMap, view.premature, view.intervalTree);
+                                    view.compactingMap, view.intervalTree);
 
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed);
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, emptySet(), flushed);
-                Set<SSTableReader> premature = replace(view.premature, emptySet(), flushed);
-                return new View(view.liveMemtables, flushingMemtables, sstableMap, compactingMap, premature,
+                return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
     }
 
-    static Function<View, View> permitCompactionOfFlushed(final Collection<SSTableReader> readers)
-    {
-        Set<SSTableReader> expectAndRemove = ImmutableSet.copyOf(readers);
-        return new Function<View, View>()
-        {
-            public View apply(View view)
-            {
-                Set<SSTableReader> premature = replace(view.premature, expectAndRemove, emptySet());
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, expectAndRemove, emptySet());
-                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap, compactingMap, premature, view.intervalTree);
-            }
-        };
-    }
-
-
     private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
     {
         return new Predicate<T>()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/Version.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
index d9e289c..96c5a6e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/Version.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -72,6 +72,8 @@ public abstract class Version
 
     public abstract boolean hasCommitLogLowerBound();
 
+    public abstract boolean hasCommitLogIntervals();
+
     public String getVersion()
     {
         return version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index e0fb3b1..16f0beb 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -111,7 +111,7 @@ public class BigFormat implements SSTableFormat
     // we always incremented the major version.
     static class BigVersion extends Version
     {
-        public static final String current_version = "mb";
+        public static final String current_version = "mc";
         public static final String earliest_supported_version = "jb";
 
         // jb (2.0.1): switch from crc32 to adler32 for compression checksums
@@ -124,7 +124,8 @@ public class BigFormat implements SSTableFormat
         // lb (2.2.7): commit log lower bound included
         // ma (3.0.0): swap bf hash order
         //             store rows natively
-        // mb (3.0.6): commit log lower bound included
+        // mb (3.0.7, 3.7): commit log lower bound included
+        // mc (3.0.8, 3.9): commit log intervals included
         //
         // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
 
@@ -145,6 +146,7 @@ public class BigFormat implements SSTableFormat
          */
         private final boolean hasOldBfHashOrder;
         private final boolean hasCommitLogLowerBound;
+        private final boolean hasCommitLogIntervals;
 
         /**
          * CASSANDRA-7066: compaction ancerstors are no longer used and have been removed.
@@ -186,6 +188,7 @@ public class BigFormat implements SSTableFormat
             hasBoundaries = version.compareTo("ma") < 0;
             hasCommitLogLowerBound = (version.compareTo("lb") >= 0 && version.compareTo("ma") < 0)
                                      || version.compareTo("mb") >= 0;
+            hasCommitLogIntervals = version.compareTo("mc") >= 0;
         }
 
         @Override
@@ -248,12 +251,19 @@ public class BigFormat implements SSTableFormat
             return newFileName;
         }
 
+        @Override
         public boolean hasCommitLogLowerBound()
         {
             return hasCommitLogLowerBound;
         }
 
         @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return hasCommitLogIntervals;
+        }
+
+        @Override
         public boolean storeRows()
         {
             return storeRows;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 4561520..a683513 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@ -24,6 +24,7 @@ import java.util.*;
 import com.google.common.collect.Maps;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
@@ -35,6 +36,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.StreamingHistogram;
 
+import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.replayPositionSetSerializer;
+
 /**
  * Serializer for SSTable from legacy versions
  */
@@ -55,7 +58,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
         EstimatedHistogram.serializer.serialize(stats.estimatedPartitionSize, out);
         EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-        ReplayPosition.serializer.serialize(stats.commitLogUpperBound, out);
+        ReplayPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
         out.writeLong(stats.minTimestamp);
         out.writeLong(stats.maxTimestamp);
         out.writeInt(stats.maxLocalDeletionTime);
@@ -72,7 +75,9 @@ public class LegacyMetadataSerializer extends MetadataSerializer
         for (ByteBuffer value : stats.maxClusteringValues)
             ByteBufferUtil.writeWithShortLength(value, out);
         if (version.hasCommitLogLowerBound())
-            ReplayPosition.serializer.serialize(stats.commitLogLowerBound, out);
+            ReplayPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+        if (version.hasCommitLogIntervals())
+            replayPositionSetSerializer.serialize(stats.commitLogIntervals, out);
     }
 
     /**
@@ -121,6 +126,11 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
                 if (descriptor.version.hasCommitLogLowerBound())
                     commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+                IntervalSet<ReplayPosition> commitLogIntervals;
+                if (descriptor.version.hasCommitLogIntervals())
+                    commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+                else
+                    commitLogIntervals = new IntervalSet<>(commitLogLowerBound, commitLogUpperBound);
 
                 if (types.contains(MetadataType.VALIDATION))
                     components.put(MetadataType.VALIDATION,
@@ -129,8 +139,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
                     components.put(MetadataType.STATS,
                                    new StatsMetadata(partitionSizes,
                                                      columnCounts,
-                                                     commitLogLowerBound,
-                                                     commitLogUpperBound,
+                                                     commitLogIntervals,
                                                      minTimestamp,
                                                      maxTimestamp,
                                                      Integer.MAX_VALUE,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 53cf0b0..1ff2ca8 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -27,12 +27,11 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
 
 import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 import com.clearspring.analytics.stream.cardinality.ICardinality;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
 import org.apache.cassandra.db.rows.Cell;
@@ -69,8 +68,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         return new StatsMetadata(defaultPartitionSizeHistogram(),
                                  defaultCellPerPartitionCountHistogram(),
-                                 ReplayPosition.NONE,
-                                 ReplayPosition.NONE,
+                                 IntervalSet.empty(),
                                  Long.MIN_VALUE,
                                  Long.MAX_VALUE,
                                  Integer.MAX_VALUE,
@@ -91,8 +89,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
     // TODO: cound the number of row per partition (either with the number of cells, or instead)
     protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
-    protected ReplayPosition commitLogLowerBound = ReplayPosition.NONE;
-    protected ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
+    protected IntervalSet commitLogIntervals = IntervalSet.empty();
     protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
     protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
     protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
@@ -126,23 +123,13 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         this(comparator);
 
-        ReplayPosition min = null, max = null;
+        IntervalSet.Builder intervals = new IntervalSet.Builder();
         for (SSTableReader sstable : sstables)
         {
-            if (min == null)
-            {
-                min = sstable.getSSTableMetadata().commitLogLowerBound;
-                max = sstable.getSSTableMetadata().commitLogUpperBound;
-            }
-            else
-            {
-                min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
-                max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
-            }
+            intervals.addAll(sstable.getSSTableMetadata().commitLogIntervals);
         }
 
-        commitLogLowerBound(min);
-        commitLogUpperBound(max);
+        commitLogIntervals(intervals.build());
         sstableLevel(level);
     }
 
@@ -229,15 +216,9 @@ public class MetadataCollector implements PartitionStatisticsCollector
         ttlTracker.update(newTTL);
     }
 
-    public MetadataCollector commitLogLowerBound(ReplayPosition commitLogLowerBound)
-    {
-        this.commitLogLowerBound = commitLogLowerBound;
-        return this;
-    }
-
-    public MetadataCollector commitLogUpperBound(ReplayPosition commitLogUpperBound)
+    public MetadataCollector commitLogIntervals(IntervalSet commitLogIntervals)
     {
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         return this;
     }
 
@@ -302,8 +283,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
         components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
         components.put(MetadataType.STATS, new StatsMetadata(estimatedPartitionSize,
                                                              estimatedCellPerPartitionCount,
-                                                             commitLogLowerBound,
-                                                             commitLogUpperBound,
+                                                             commitLogIntervals,
                                                              timestampTracker.min(),
                                                              timestampTracker.max(),
                                                              localDeletionTimeTracker.min(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index 07e35bb..9971eaa 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@ -22,10 +22,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -39,11 +41,11 @@ import org.apache.cassandra.utils.StreamingHistogram;
 public class StatsMetadata extends MetadataComponent
 {
     public static final IMetadataComponentSerializer serializer = new StatsMetadataSerializer();
+    public static final ISerializer<IntervalSet<ReplayPosition>> replayPositionSetSerializer = IntervalSet.serializer(ReplayPosition.serializer);
 
     public final EstimatedHistogram estimatedPartitionSize;
     public final EstimatedHistogram estimatedColumnCount;
-    public final ReplayPosition commitLogLowerBound;
-    public final ReplayPosition commitLogUpperBound;
+    public final IntervalSet<ReplayPosition> commitLogIntervals;
     public final long minTimestamp;
     public final long maxTimestamp;
     public final int minLocalDeletionTime;
@@ -62,8 +64,7 @@ public class StatsMetadata extends MetadataComponent
 
     public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                          EstimatedHistogram estimatedColumnCount,
-                         ReplayPosition commitLogLowerBound,
-                         ReplayPosition commitLogUpperBound,
+                         IntervalSet<ReplayPosition> commitLogIntervals,
                          long minTimestamp,
                          long maxTimestamp,
                          int minLocalDeletionTime,
@@ -82,8 +83,7 @@ public class StatsMetadata extends MetadataComponent
     {
         this.estimatedPartitionSize = estimatedPartitionSize;
         this.estimatedColumnCount = estimatedColumnCount;
-        this.commitLogLowerBound = commitLogLowerBound;
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         this.minTimestamp = minTimestamp;
         this.maxTimestamp = maxTimestamp;
         this.minLocalDeletionTime = minLocalDeletionTime;
@@ -134,8 +134,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -157,8 +156,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -186,8 +184,7 @@ public class StatsMetadata extends MetadataComponent
         return new EqualsBuilder()
                        .append(estimatedPartitionSize, that.estimatedPartitionSize)
                        .append(estimatedColumnCount, that.estimatedColumnCount)
-                       .append(commitLogLowerBound, that.commitLogLowerBound)
-                       .append(commitLogUpperBound, that.commitLogUpperBound)
+                       .append(commitLogIntervals, that.commitLogIntervals)
                        .append(minTimestamp, that.minTimestamp)
                        .append(maxTimestamp, that.maxTimestamp)
                        .append(minLocalDeletionTime, that.minLocalDeletionTime)
@@ -212,8 +209,7 @@ public class StatsMetadata extends MetadataComponent
         return new HashCodeBuilder()
                        .append(estimatedPartitionSize)
                        .append(estimatedColumnCount)
-                       .append(commitLogLowerBound)
-                       .append(commitLogUpperBound)
+                       .append(commitLogIntervals)
                        .append(minTimestamp)
                        .append(maxTimestamp)
                        .append(minLocalDeletionTime)
@@ -239,7 +235,7 @@ public class StatsMetadata extends MetadataComponent
             int size = 0;
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize);
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount);
-            size += ReplayPosition.serializer.serializedSize(component.commitLogUpperBound);
+            size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE));
             if (version.storeRows())
                 size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long)
             else
@@ -258,7 +254,9 @@ public class StatsMetadata extends MetadataComponent
             if (version.storeRows())
                 size += 8 + 8; // totalColumnsSet, totalRows
             if (version.hasCommitLogLowerBound())
-                size += ReplayPosition.serializer.serializedSize(component.commitLogLowerBound);
+                size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE));
+            if (version.hasCommitLogIntervals())
+                size += replayPositionSetSerializer.serializedSize(component.commitLogIntervals);
             return size;
         }
 
@@ -266,7 +264,7 @@ public class StatsMetadata extends MetadataComponent
         {
             EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out);
             EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-            ReplayPosition.serializer.serialize(component.commitLogUpperBound, out);
+            ReplayPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
             out.writeLong(component.minTimestamp);
             out.writeLong(component.maxTimestamp);
             if (version.storeRows())
@@ -296,7 +294,9 @@ public class StatsMetadata extends MetadataComponent
             }
 
             if (version.hasCommitLogLowerBound())
-                ReplayPosition.serializer.serialize(component.commitLogLowerBound, out);
+                ReplayPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+            if (version.hasCommitLogIntervals())
+                replayPositionSetSerializer.serialize(component.commitLogIntervals, out);
         }
 
         public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@ -338,11 +338,15 @@ public class StatsMetadata extends MetadataComponent
 
             if (version.hasCommitLogLowerBound())
                 commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+            IntervalSet<ReplayPosition> commitLogIntervals;
+            if (version.hasCommitLogIntervals())
+                commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+            else
+                commitLogIntervals = new IntervalSet<ReplayPosition>(commitLogLowerBound, commitLogUpperBound);
 
             return new StatsMetadata(partitionSizes,
                                      columnCounts,
-                                     commitLogLowerBound,
-                                     commitLogUpperBound,
+                                     commitLogIntervals,
                                      minTimestamp,
                                      maxTimestamp,
                                      minLocalDeletionTime,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 420b802..5f7513f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -70,8 +70,7 @@ public class SSTableMetadataViewer
                     out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                     out.printf("SSTable Level: %d%n", stats.sstableLevel);
                     out.printf("Repaired at: %d%n", stats.repairedAt);
-                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
-                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                     out.println("Estimated tombstone drop times:");
                     for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                     {


[14/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
Change commitlog and sstables to track dirty and clean intervals.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828


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

Branch: refs/heads/trunk
Commit: 904cb5d10e0de1a6ca89249be8c257ed38a80ef0
Parents: cf85f52
Author: Branimir Lambov <br...@datastax.com>
Authored: Sat May 14 11:31:16 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:38:37 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  66 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  18 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         |  59 +++-
 .../db/commitlog/CommitLogSegment.java          |  77 ++---
 .../db/commitlog/CommitLogSegmentManager.java   |   4 +-
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../cassandra/db/commitlog/ReplayPosition.java  |  71 ----
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  44 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  36 +-
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  14 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  38 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   2 +-
 test/unit/org/apache/cassandra/Util.java        |  21 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 159 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 98 files changed, 1229 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 046c8b3..b596fc9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
  * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
  * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
  * Fix upgrade of super columns on thrift (CASSANDRA-12335)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
index f47fd57..bc733d7 100644
--- a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
+++ b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
@@ -29,6 +29,8 @@ import java.util.concurrent.CopyOnWriteArraySet;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
@@ -101,6 +103,17 @@ public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
     }
 
     /**
+     * Testing only!
+     * Clear the set of unwritable directories.
+     */
+    @VisibleForTesting
+    public static void clearUnwritableUnsafe()
+    {
+        instance.unwritableDirectories.clear();
+    }
+
+
+    /**
      * Tells whether or not the directory is blacklisted for reads.
      * @return whether or not the directory is blacklisted for reads.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 400fd36..82604e2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -179,9 +179,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    public static volatile ColumnFamilyStore discardFlushResults;
-
     public final Keyspace keyspace;
     public final String name;
     public final CFMetaData metadata;
@@ -926,25 +923,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         volatile FSWriteError flushFailure = null;
-        final ReplayPosition commitLogUpperBound;
         final List<Memtable> memtables;
-        final List<Collection<SSTableReader>> readers;
 
-        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
-                          List<Memtable> memtables, List<Collection<SSTableReader>> readers)
+        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier,
+                          List<Memtable> memtables)
         {
             this.writeBarrier = writeBarrier;
             this.flushSecondaryIndexes = flushSecondaryIndexes;
-            this.commitLogUpperBound = commitLogUpperBound;
             this.memtables = memtables;
-            this.readers = readers;
         }
 
         public ReplayPosition call()
         {
-            if (discardFlushResults == ColumnFamilyStore.this)
-                return commitLogUpperBound;
-
             writeBarrier.await();
 
             /**
@@ -968,17 +958,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 throw new IllegalStateException();
             }
 
+            ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (flushFailure == null)
+            if (flushFailure == null && !memtables.isEmpty())
             {
-                CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                for (int i = 0 ; i < memtables.size() ; i++)
-                {
-                    Memtable memtable = memtables.get(i);
-                    Collection<SSTableReader> reader = readers.get(i);
-                    memtable.cfs.data.permitCompactionOfFlushed(reader);
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, reader);
-                }
+                Memtable memtable = memtables.get(0);
+                commitLogUpperBound = memtable.getCommitLogUpperBound();
+                CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
             }
 
             metric.pendingFlushes.dec();
@@ -1002,7 +988,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         final OpOrder.Barrier writeBarrier;
         final List<Memtable> memtables = new ArrayList<>();
-        final List<Collection<SSTableReader>> readers = new ArrayList<>();
         final PostFlush postFlush;
         final boolean truncate;
 
@@ -1044,7 +1029,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
             // replay positions have also completed, i.e. the memtables are done and ready to flush
             writeBarrier.issue();
-            postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
+            postFlush = new PostFlush(!truncate, writeBarrier, memtables);
         }
 
         public void run()
@@ -1063,8 +1048,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 memtable.cfs.data.markFlushing(memtable);
                 if (memtable.isClean() || truncate)
                 {
-                    memtable.cfs.data.replaceFlushed(memtable, Collections.emptyList());
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, Collections.emptyList());
+                    memtable.cfs.replaceFlushed(memtable, Collections.emptyList());
                     reclaim(memtable);
                     iter.remove();
                 }
@@ -1077,9 +1061,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (Memtable memtable : memtables)
                 {
                     Collection<SSTableReader> readers = memtable.flush();
-                    memtable.cfs.data.replaceFlushed(memtable, readers);
+                    memtable.cfs.replaceFlushed(memtable, readers);
                     reclaim(memtable);
-                    this.readers.add(readers);
                 }
             }
             catch (FSWriteError e)
@@ -1126,21 +1109,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    // this method should ONLY be used for testing commit log behaviour; it discards the current memtable
-    // contents without marking the commit log clean, and prevents any proceeding flushes from marking
-    // the commit log as done, however they *will* terminate (unlike under typical failures) to ensure progress is made
-    public void simulateFailedFlush()
-    {
-        discardFlushResults = this;
-        data.markFlushing(data.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this)));
-    }
-
-    public void resumeFlushing()
-    {
-        discardFlushResults = null;
-    }
-
     /**
      * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
      * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
@@ -1483,16 +1451,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return data;
     }
 
-    public Collection<SSTableReader> getSSTables()
-    {
-        return data.getSSTables();
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompactSSTables()
-    {
-        return data.getPermittedToCompact();
-    }
-
     public Set<SSTableReader> getLiveSSTables()
     {
         return data.getView().liveSSTables();
@@ -2032,7 +1990,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         long now = System.currentTimeMillis();
         // make sure none of our sstables are somehow in the future (clock drift, perhaps)
         for (ColumnFamilyStore cfs : concatWithIndexes())
-            for (SSTableReader sstable : cfs.data.getSSTables())
+            for (SSTableReader sstable : cfs.getLiveSSTables())
                 now = Math.max(now, sstable.maxDataAge);
         truncatedAt = now;
 
@@ -2130,7 +2088,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             public LifecycleTransaction call() throws Exception
             {
                 assert data.getCompacting().isEmpty() : data.getCompacting();
-                Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                Iterable<SSTableReader> sstables = getLiveSSTables();
                 sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
                 sstables = ImmutableList.copyOf(sstables);
                 LifecycleTransaction modifier = data.tryModify(sstables, operationType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 01ffd52..877f984 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -385,7 +385,7 @@ public class Directories
 
         if (candidates.isEmpty())
             if (tooBig)
-                throw new RuntimeException("Insufficient disk space to write " + writeSize + " bytes");
+                throw new FSWriteError(new IOException("Insufficient disk space to write " + writeSize + " bytes"), "");
             else
                 throw new FSWriteError(new IOException("All configured data directories have been blacklisted as unwritable for erroring out"), "");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 93dc5af..3c77092 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
@@ -193,6 +194,11 @@ public class Memtable implements Comparable<Memtable>
         return commitLogLowerBound.get();
     }
 
+    public ReplayPosition getCommitLogUpperBound()
+    {
+        return commitLogUpperBound.get();
+    }
+
     public boolean isLive()
     {
         return allocator.isLive();
@@ -331,6 +337,15 @@ public class Memtable implements Comparable<Memtable>
         return minTimestamp;
     }
 
+    /**
+     * For testing only. Give this memtable too big a size to make it always fail flushing.
+     */
+    @VisibleForTesting
+    public void makeUnflushable()
+    {
+        liveDataSize.addAndGet(1L * 1024 * 1024 * 1024 * 1024 * 1024);
+    }
+
     private long estimatedSize()
     {
         long keySize = 0;
@@ -418,8 +433,7 @@ public class Memtable implements Comparable<Memtable>
         {
             txn = LifecycleTransaction.offline(OperationType.FLUSH);
             MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
-                                                         .commitLogLowerBound(commitLogLowerBound.get())
-                                                         .commitLogUpperBound(commitLogUpperBound.get());
+                    .commitLogIntervals(new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));
 
             return new SSTableTxnWriter(txn,
                                         cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index dcdd855..dfe3f91 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -290,11 +290,12 @@ public class CommitLog implements CommitLogMBean
      * given. Discards any commit log segments that are no longer used.
      *
      * @param cfId    the column family ID that was flushed
-     * @param context the replay position of the flush
+     * @param lowerBound the lowest covered replay position of the flush
+     * @param lowerBound the highest covered replay position of the flush
      */
-    public void discardCompletedSegments(final UUID cfId, final ReplayPosition context)
+    public void discardCompletedSegments(final UUID cfId, final ReplayPosition lowerBound, final ReplayPosition upperBound)
     {
-        logger.trace("discard completed log segments for {}, table {}", context, cfId);
+        logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
@@ -303,7 +304,7 @@ public class CommitLog implements CommitLogMBean
         for (Iterator<CommitLogSegment> iter = allocator.getActiveSegments().iterator(); iter.hasNext();)
         {
             CommitLogSegment segment = iter.next();
-            segment.markClean(cfId, context);
+            segment.markClean(cfId, lowerBound, upperBound);
 
             if (segment.isUnused())
             {
@@ -318,7 +319,7 @@ public class CommitLog implements CommitLogMBean
 
             // Don't mark or try to delete any newer segments once we've reached the one containing the
             // position of the flush.
-            if (segment.contains(context))
+            if (segment.contains(upperBound))
                 break;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index f45a47a..af8efb4 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -35,6 +35,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.commons.lang3.StringUtils;
@@ -52,6 +53,7 @@ import org.apache.cassandra.io.util.FileSegmentInputStream;
 import org.apache.cassandra.io.util.RebufferingInputStream;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.FileDataInput;
@@ -74,7 +76,7 @@ public class CommitLogReplayer
     private final List<Future<?>> futures;
     private final Map<UUID, AtomicInteger> invalidMutations;
     private final AtomicInteger replayedCount;
-    private final Map<UUID, ReplayPosition.ReplayFilter> cfPersisted;
+    private final Map<UUID, IntervalSet<ReplayPosition>> cfPersisted;
     private final ReplayPosition globalPosition;
     private final CRC32 checksum;
     private byte[] buffer;
@@ -83,7 +85,7 @@ public class CommitLogReplayer
     private final ReplayFilter replayFilter;
     private final CommitLogArchiver archiver;
 
-    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, ReplayPosition.ReplayFilter> cfPersisted, ReplayFilter replayFilter)
+    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, IntervalSet<ReplayPosition>> cfPersisted, ReplayFilter replayFilter)
     {
         this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
@@ -101,10 +103,9 @@ public class CommitLogReplayer
 
     public static CommitLogReplayer construct(CommitLog commitLog)
     {
-        // compute per-CF and global replay positions
-        Map<UUID, ReplayPosition.ReplayFilter> cfPersisted = new HashMap<>();
+        // compute per-CF and global replay intervals
+        Map<UUID, IntervalSet<ReplayPosition>> cfPersisted = new HashMap<>();
         ReplayFilter replayFilter = ReplayFilter.create();
-        ReplayPosition globalPosition = null;
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
@@ -129,14 +130,10 @@ public class CommitLogReplayer
                 }
             }
 
-            ReplayPosition.ReplayFilter filter = new ReplayPosition.ReplayFilter(cfs.getSSTables(), truncatedAt);
-            if (!filter.isEmpty())
-                cfPersisted.put(cfs.metadata.cfId, filter);
-            else
-                globalPosition = ReplayPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
+            IntervalSet<ReplayPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
+            cfPersisted.put(cfs.metadata.cfId, filter);
         }
-        if (globalPosition == null)
-            globalPosition = ReplayPosition.firstNotCovered(cfPersisted.values());
+        ReplayPosition globalPosition = firstNotCovered(cfPersisted.values());
         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
         return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
     }
@@ -148,6 +145,41 @@ public class CommitLogReplayer
             recover(clogs[i], i + 1 == clogs.length);
     }
 
+    /**
+     * A set of known safe-to-discard commit log replay positions, based on
+     * the range covered by on disk sstables and those prior to the most recent truncation record
+     */
+    public static IntervalSet<ReplayPosition> persistedIntervals(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
+    {
+        IntervalSet.Builder<ReplayPosition> builder = new IntervalSet.Builder<>();
+        for (SSTableReader reader : onDisk)
+            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+
+        if (truncatedAt != null)
+            builder.add(ReplayPosition.NONE, truncatedAt);
+        return builder.build();
+    }
+
+    /**
+     * Find the earliest commit log position that is not covered by the known flushed ranges for some table.
+     *
+     * For efficiency this assumes that the first contiguously flushed interval we know of contains the moment that the
+     * given table was constructed* and hence we can start replay from the end of that interval.
+     *
+     * If such an interval is not known, we must replay from the beginning.
+     *
+     * * This is not true only until if the very first flush of a table stalled or failed, while the second or latter
+     *   succeeded. The chances of this happening are at most very low, and if the assumption does prove to be
+     *   incorrect during replay there is little chance that the affected deployment is in production.
+     */
+    public static ReplayPosition firstNotCovered(Collection<IntervalSet<ReplayPosition>> ranges)
+    {
+        return ranges.stream()
+                .map(intervals -> Iterables.getFirst(intervals.ends(), ReplayPosition.NONE)) 
+                .min(Ordering.natural())
+                .get(); // iteration is per known-CF, there must be at least one. 
+    }
+
     public int blockForWrites()
     {
         for (Map.Entry<UUID, AtomicInteger> entry : invalidMutations.entrySet())
@@ -293,8 +325,7 @@ public class CommitLogReplayer
      */
     private boolean shouldReplay(UUID cfId, ReplayPosition position)
     {
-        ReplayPosition.ReplayFilter filter = cfPersisted.get(cfId);
-        return filter == null || filter.shouldReplay(position);
+        return !cfPersisted.get(cfId).contains(position);
     }
 
     @SuppressWarnings("resource")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 27c05b4..d2f12bf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -101,11 +102,11 @@ public abstract class CommitLogSegment
     // a signal for writers to wait on to confirm the log message they provided has been written to disk
     private final WaitQueue syncComplete = new WaitQueue();
 
-    // a map of Cf->dirty position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final NonBlockingHashMap<UUID, AtomicInteger> cfDirty = new NonBlockingHashMap<>(1024);
+    // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
+    private final NonBlockingHashMap<UUID, IntegerInterval> cfDirty = new NonBlockingHashMap<>(1024);
 
-    // a map of Cf->clean position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final ConcurrentHashMap<UUID, AtomicInteger> cfClean = new ConcurrentHashMap<>();
+    // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use
+    private final ConcurrentHashMap<UUID, IntegerInterval.Set> cfClean = new ConcurrentHashMap<>();
 
     public final long id;
 
@@ -423,10 +424,23 @@ public abstract class CommitLogSegment
         }
     }
 
+    public static<K> void coverInMap(ConcurrentMap<K, IntegerInterval> map, K key, int value)
+    {
+        IntegerInterval i = map.get(key);
+        if (i == null)
+        {
+            i = map.putIfAbsent(key, new IntegerInterval(value, value));
+            if (i == null)
+                // success
+                return;
+        }
+        i.expandToCover(value);
+    }
+
     void markDirty(Mutation mutation, int allocatedPosition)
     {
         for (PartitionUpdate update : mutation.getPartitionUpdates())
-            ensureAtleast(cfDirty, update.metadata().cfId, allocatedPosition);
+            coverInMap(cfDirty, update.metadata().cfId, allocatedPosition);
     }
 
     /**
@@ -437,55 +451,32 @@ public abstract class CommitLogSegment
      * @param cfId    the column family ID that is now clean
      * @param context the optional clean offset
      */
-    public synchronized void markClean(UUID cfId, ReplayPosition context)
+    public synchronized void markClean(UUID cfId, ReplayPosition startPosition, ReplayPosition endPosition)
     {
+        if (startPosition.segment > id || endPosition.segment < id)
+            return;
         if (!cfDirty.containsKey(cfId))
             return;
-        if (context.segment == id)
-            markClean(cfId, context.position);
-        else if (context.segment > id)
-            markClean(cfId, Integer.MAX_VALUE);
-    }
-
-    private void markClean(UUID cfId, int position)
-    {
-        ensureAtleast(cfClean, cfId, position);
+        int start = startPosition.segment == id ? startPosition.position : 0;
+        int end = endPosition.segment == id ? endPosition.position : Integer.MAX_VALUE;
+        cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
         removeCleanFromDirty();
     }
 
-    private static void ensureAtleast(ConcurrentMap<UUID, AtomicInteger> map, UUID cfId, int value)
-    {
-        AtomicInteger i = map.get(cfId);
-        if (i == null)
-        {
-            AtomicInteger i2 = map.putIfAbsent(cfId, i = new AtomicInteger());
-            if (i2 != null)
-                i = i2;
-        }
-        while (true)
-        {
-            int cur = i.get();
-            if (cur > value)
-                break;
-            if (i.compareAndSet(cur, value))
-                break;
-        }
-    }
-
     private void removeCleanFromDirty()
     {
         // if we're still allocating from this segment, don't touch anything since it can't be done thread-safely
         if (isStillAllocating())
             return;
 
-        Iterator<Map.Entry<UUID, AtomicInteger>> iter = cfClean.entrySet().iterator();
+        Iterator<Map.Entry<UUID, IntegerInterval.Set>> iter = cfClean.entrySet().iterator();
         while (iter.hasNext())
         {
-            Map.Entry<UUID, AtomicInteger> clean = iter.next();
+            Map.Entry<UUID, IntegerInterval.Set> clean = iter.next();
             UUID cfId = clean.getKey();
-            AtomicInteger cleanPos = clean.getValue();
-            AtomicInteger dirtyPos = cfDirty.get(cfId);
-            if (dirtyPos != null && dirtyPos.intValue() <= cleanPos.intValue())
+            IntegerInterval.Set cleanSet = clean.getValue();
+            IntegerInterval dirtyInterval = cfDirty.get(cfId);
+            if (dirtyInterval != null && cleanSet.covers(dirtyInterval))
             {
                 cfDirty.remove(cfId);
                 iter.remove();
@@ -502,12 +493,12 @@ public abstract class CommitLogSegment
             return cfDirty.keySet();
 
         List<UUID> r = new ArrayList<>(cfDirty.size());
-        for (Map.Entry<UUID, AtomicInteger> dirty : cfDirty.entrySet())
+        for (Map.Entry<UUID, IntegerInterval> dirty : cfDirty.entrySet())
         {
             UUID cfId = dirty.getKey();
-            AtomicInteger dirtyPos = dirty.getValue();
-            AtomicInteger cleanPos = cfClean.get(cfId);
-            if (cleanPos == null || cleanPos.intValue() < dirtyPos.intValue())
+            IntegerInterval dirtyInterval = dirty.getValue();
+            IntegerInterval.Set cleanSet = cfClean.get(cfId);
+            if (cleanSet == null || !cleanSet.covers(dirtyInterval))
                 r.add(dirty.getKey());
         }
         return r;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 66ad6a3..82cee50 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -310,7 +310,7 @@ public class CommitLogSegmentManager
 
             for (CommitLogSegment segment : activeSegments)
                 for (UUID cfId : droppedCfs)
-                    segment.markClean(cfId, segment.getContext());
+                    segment.markClean(cfId, ReplayPosition.NONE, segment.getContext());
 
             // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
             // if the previous active segment was the only one to recycle (since an active segment isn't
@@ -451,7 +451,7 @@ public class CommitLogSegmentManager
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
                     logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                    segment.markClean(dirtyCFId, segment.getContext());
+                    segment.markClean(dirtyCFId, ReplayPosition.NONE, segment.getContext());
                 }
                 else if (!flushes.containsKey(dirtyCFId))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
new file mode 100644
index 0000000..bd0ea22
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
@@ -0,0 +1,192 @@
+package org.apache.cassandra.db.commitlog;
+
+import java.io.IOException;
+import java.util.*;
+
+import com.google.common.collect.ImmutableSortedMap;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An immutable set of closed intervals, stored in normalized form (i.e. where overlapping intervals are converted
+ * to a single interval covering both).
+ *
+ * The set is stored as a sorted map from interval starts to the corresponding end. The map satisfies
+ *   curr().getKey() <= curr().getValue() < next().getKey()
+ */
+public class IntervalSet<T extends Comparable<T>>
+{
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static final IntervalSet EMPTY = new IntervalSet(ImmutableSortedMap.of());
+
+    final private NavigableMap<T, T> ranges;
+
+    private IntervalSet(ImmutableSortedMap<T, T> ranges)
+    {
+        this.ranges = ranges;
+    }
+
+    /**
+     * Construct new set containing the interval with the given start and end position.
+     */
+    public IntervalSet(T start, T end)
+    {
+        this(ImmutableSortedMap.of(start, end));
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T extends Comparable<T>> IntervalSet<T> empty()
+    {
+        return (IntervalSet<T>) EMPTY;
+    }
+
+    public boolean contains(T position)
+    {
+        // closed (i.e. inclusive) intervals
+        Map.Entry<T, T> range = ranges.floorEntry(position);
+        return range != null && position.compareTo(range.getValue()) <= 0;
+    }
+
+    public boolean isEmpty()
+    {
+        return ranges.isEmpty();
+    }
+
+    public Optional<T> lowerBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.firstKey());
+    }
+
+    public Optional<T> upperBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.lastEntry().getValue());
+    }
+
+    public Collection<T> starts()
+    {
+        return ranges.keySet();
+    }
+
+    public Collection<T> ends()
+    {
+        return ranges.values();
+    }
+
+    public String toString()
+    {
+        return ranges.toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return ranges.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        return obj instanceof IntervalSet && ranges.equals(((IntervalSet<?>) obj).ranges);
+    }
+
+    public static final <T extends Comparable<T>> ISerializer<IntervalSet<T>> serializer(ISerializer<T> pointSerializer)
+    {
+        return new ISerializer<IntervalSet<T>>()
+        {
+            public void serialize(IntervalSet<T> intervals, DataOutputPlus out) throws IOException
+            {
+                out.writeInt(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    pointSerializer.serialize(en.getKey(), out);
+                    pointSerializer.serialize(en.getValue(), out);
+                }
+            }
+    
+            public IntervalSet<T> deserialize(DataInputPlus in) throws IOException
+            {
+                int count = in.readInt();
+                NavigableMap<T, T> ranges = new TreeMap<>();
+                for (int i = 0; i < count; ++i)
+                    ranges.put(pointSerializer.deserialize(in), pointSerializer.deserialize(in));
+                return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+            }
+    
+            public long serializedSize(IntervalSet<T> intervals)
+            {
+                long size = TypeSizes.sizeof(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    size += pointSerializer.serializedSize(en.getKey());
+                    size += pointSerializer.serializedSize(en.getValue());
+                }
+                return size;
+            }
+        };
+    };
+
+    /**
+     * Builder of interval sets, applying the necessary normalization while adding ranges.
+     *
+     * Data is stored as above, as a sorted map from interval starts to the corresponding end, which satisfies
+     *   curr().getKey() <= curr().getValue() < next().getKey()
+     */
+    static public class Builder<T extends Comparable<T>>
+    {
+        final NavigableMap<T, T> ranges;
+
+        public Builder()
+        {
+            this.ranges = new TreeMap<>();
+        }
+
+        public Builder(T start, T end)
+        {
+            this();
+            assert start.compareTo(end) <= 0;
+            ranges.put(start, end);
+        }
+
+        /**
+         * Add an interval to the set and perform normalization.
+         */
+        public void add(T start, T end)
+        {
+            assert start.compareTo(end) <= 0;
+            // extend ourselves to cover any ranges we overlap
+            // record directly preceding our end may extend past us, so take the max of our end and its
+            Map.Entry<T, T> extend = ranges.floorEntry(end);
+            if (extend != null && extend.getValue().compareTo(end) > 0)
+                end = extend.getValue();
+
+            // record directly preceding our start may extend into us; if it does, we take it as our start
+            extend = ranges.lowerEntry(start);
+            if (extend != null && extend.getValue().compareTo(start) >= 0)
+                start = extend.getKey();
+
+            // remove all covered intervals
+            // since we have adjusted start and end to cover the ones that would be only partially covered, we
+            // are certain that anything whose start falls within the span is completely covered
+            ranges.subMap(start, end).clear();
+            // add the new interval
+            ranges.put(start, end);
+        }
+
+        public void addAll(IntervalSet<T> otherSet)
+        {
+            for (Map.Entry<T, T> en : otherSet.ranges.entrySet())
+            {
+                add(en.getKey(), en.getValue());
+            }
+        }
+
+        public IntervalSet<T> build()
+        {
+            return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
index 0b21763..b0214b8 100644
--- a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
+++ b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
@@ -18,15 +18,9 @@
 package org.apache.cassandra.db.commitlog;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
@@ -43,71 +37,6 @@ public class ReplayPosition implements Comparable<ReplayPosition>
     public final long segment;
     public final int position;
 
-    /**
-     * A filter of known safe-to-discard commit log replay positions, based on
-     * the range covered by on disk sstables and those prior to the most recent truncation record
-     */
-    public static class ReplayFilter
-    {
-        final NavigableMap<ReplayPosition, ReplayPosition> persisted = new TreeMap<>();
-        public ReplayFilter(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
-        {
-            for (SSTableReader reader : onDisk)
-            {
-                ReplayPosition start = reader.getSSTableMetadata().commitLogLowerBound;
-                ReplayPosition end = reader.getSSTableMetadata().commitLogUpperBound;
-                add(persisted, start, end);
-            }
-            if (truncatedAt != null)
-                add(persisted, ReplayPosition.NONE, truncatedAt);
-        }
-
-        private static void add(NavigableMap<ReplayPosition, ReplayPosition> ranges, ReplayPosition start, ReplayPosition end)
-        {
-            // extend ourselves to cover any ranges we overlap
-            // record directly preceding our end may extend past us, so take the max of our end and its
-            Map.Entry<ReplayPosition, ReplayPosition> extend = ranges.floorEntry(end);
-            if (extend != null && extend.getValue().compareTo(end) > 0)
-                end = extend.getValue();
-
-            // record directly preceding our start may extend into us; if it does, we take it as our start
-            extend = ranges.lowerEntry(start);
-            if (extend != null && extend.getValue().compareTo(start) >= 0)
-                start = extend.getKey();
-
-            ranges.subMap(start, end).clear();
-            ranges.put(start, end);
-        }
-
-        public boolean shouldReplay(ReplayPosition position)
-        {
-            // replay ranges are start exclusive, end inclusive
-            Map.Entry<ReplayPosition, ReplayPosition> range = persisted.lowerEntry(position);
-            return range == null || position.compareTo(range.getValue()) > 0;
-        }
-
-        public boolean isEmpty()
-        {
-            return persisted.isEmpty();
-        }
-    }
-
-    public static ReplayPosition firstNotCovered(Iterable<ReplayFilter> ranges)
-    {
-        ReplayPosition min = null;
-        for (ReplayFilter map : ranges)
-        {
-            ReplayPosition first = map.persisted.firstEntry().getValue();
-            if (min == null)
-                min = first;
-            else
-                min = Ordering.natural().min(min, first);
-        }
-        if (min == null)
-            return NONE;
-        return min;
-    }
-
     public ReplayPosition(long segment, int position)
     {
         this.segment = segment;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 0dce52b..a80a6f4 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -241,6 +241,9 @@ public abstract class AbstractCompactionStrategy
      */
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 444d43d..a9bfbd2 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -189,6 +189,9 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     public int getUnleveledSSTables()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index c94b88f..5a3d524 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -37,7 +37,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
@@ -48,8 +47,6 @@ import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static com.google.common.base.Predicates.and;
-import static com.google.common.base.Predicates.in;
-import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
@@ -204,7 +201,6 @@ public class Tracker
                          ImmutableList.<Memtable>of(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
                          SSTableIntervalTree.empty()));
     }
 
@@ -351,49 +347,19 @@ public class Tracker
 
         Throwable fail;
         fail = updateSizeTracking(emptySet(), sstables, null);
+        // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+        fail = notifyAdded(sstables, fail);
+
+        if (!isDummy() && !cfstore.isValid())
+            dropSSTables();
 
         maybeFail(fail);
     }
 
-    /**
-     * permit compaction of the provided sstable; this translates to notifying compaction
-     * strategies of its existence, and potentially submitting a background task
-     */
-    public void permitCompactionOfFlushed(Collection<SSTableReader> sstables)
-    {
-        if (sstables.isEmpty())
-            return;
-
-        apply(View.permitCompactionOfFlushed(sstables));
-
-        if (isDummy())
-            return;
-
-        if (cfstore.isValid())
-        {
-            notifyAdded(sstables);
-            CompactionManager.instance.submitBackground(cfstore);
-        }
-        else
-        {
-            dropSSTables();
-        }
-    }
 
 
     // MISCELLANEOUS public utility calls
 
-    public Set<SSTableReader> getSSTables()
-    {
-        return view.get().sstables;
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompact()
-    {
-        View view = this.view.get();
-        return filter(view.sstables, not(in(view.premature)));
-    }
-
     public Set<SSTableReader> getCompacting()
     {
         return view.get().compacting;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index 3fa197f..4b3aae0 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -33,7 +33,6 @@ 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.in;
 import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableList.copyOf;
 import static com.google.common.collect.ImmutableList.of;
@@ -70,7 +69,6 @@ public class View
     public final List<Memtable> flushingMemtables;
     final Set<SSTableReader> compacting;
     final Set<SSTableReader> sstables;
-    final Set<SSTableReader> premature;
     // 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
@@ -80,7 +78,7 @@ public class View
 
     final SSTableIntervalTree intervalTree;
 
-    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, Set<SSTableReader> premature, SSTableIntervalTree intervalTree)
+    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, SSTableIntervalTree intervalTree)
     {
         assert liveMemtables != null;
         assert flushingMemtables != null;
@@ -95,7 +93,6 @@ public class View
         this.sstables = sstablesMap.keySet();
         this.compactingMap = compacting;
         this.compacting = compactingMap.keySet();
-        this.premature = premature;
         this.intervalTree = intervalTree;
     }
 
@@ -256,7 +253,7 @@ public class View
                 assert all(mark, Helpers.idIn(view.sstablesMap));
                 return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
                                 replace(view.compactingMap, unmark, mark),
-                                view.premature, view.intervalTree);
+                                view.intervalTree);
             }
         };
     }
@@ -270,7 +267,7 @@ public class View
             public boolean apply(View view)
             {
                 for (SSTableReader reader : readers)
-                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted() || view.premature.contains(reader))
+                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted())
                         return false;
                 return true;
             }
@@ -287,7 +284,7 @@ public class View
             public View apply(View view)
             {
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
-                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap, view.premature,
+                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
@@ -302,7 +299,7 @@ public class 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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -321,7 +318,7 @@ public class View
                                                            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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, newFlushing, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -338,32 +335,15 @@ public class View
 
                 if (flushed == null || flushed.isEmpty())
                     return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
-                                    view.compactingMap, view.premature, view.intervalTree);
+                                    view.compactingMap, view.intervalTree);
 
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed);
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, emptySet(), flushed);
-                Set<SSTableReader> premature = replace(view.premature, emptySet(), flushed);
-                return new View(view.liveMemtables, flushingMemtables, sstableMap, compactingMap, premature,
+                return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
     }
 
-    static Function<View, View> permitCompactionOfFlushed(final Collection<SSTableReader> readers)
-    {
-        Set<SSTableReader> expectAndRemove = ImmutableSet.copyOf(readers);
-        return new Function<View, View>()
-        {
-            public View apply(View view)
-            {
-                Set<SSTableReader> premature = replace(view.premature, expectAndRemove, emptySet());
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, expectAndRemove, emptySet());
-                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap, compactingMap, premature, view.intervalTree);
-            }
-        };
-    }
-
-
     private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
     {
         return new Predicate<T>()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/Version.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
index d9e289c..96c5a6e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/Version.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -72,6 +72,8 @@ public abstract class Version
 
     public abstract boolean hasCommitLogLowerBound();
 
+    public abstract boolean hasCommitLogIntervals();
+
     public String getVersion()
     {
         return version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index e0fb3b1..16f0beb 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -111,7 +111,7 @@ public class BigFormat implements SSTableFormat
     // we always incremented the major version.
     static class BigVersion extends Version
     {
-        public static final String current_version = "mb";
+        public static final String current_version = "mc";
         public static final String earliest_supported_version = "jb";
 
         // jb (2.0.1): switch from crc32 to adler32 for compression checksums
@@ -124,7 +124,8 @@ public class BigFormat implements SSTableFormat
         // lb (2.2.7): commit log lower bound included
         // ma (3.0.0): swap bf hash order
         //             store rows natively
-        // mb (3.0.6): commit log lower bound included
+        // mb (3.0.7, 3.7): commit log lower bound included
+        // mc (3.0.8, 3.9): commit log intervals included
         //
         // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
 
@@ -145,6 +146,7 @@ public class BigFormat implements SSTableFormat
          */
         private final boolean hasOldBfHashOrder;
         private final boolean hasCommitLogLowerBound;
+        private final boolean hasCommitLogIntervals;
 
         /**
          * CASSANDRA-7066: compaction ancerstors are no longer used and have been removed.
@@ -186,6 +188,7 @@ public class BigFormat implements SSTableFormat
             hasBoundaries = version.compareTo("ma") < 0;
             hasCommitLogLowerBound = (version.compareTo("lb") >= 0 && version.compareTo("ma") < 0)
                                      || version.compareTo("mb") >= 0;
+            hasCommitLogIntervals = version.compareTo("mc") >= 0;
         }
 
         @Override
@@ -248,12 +251,19 @@ public class BigFormat implements SSTableFormat
             return newFileName;
         }
 
+        @Override
         public boolean hasCommitLogLowerBound()
         {
             return hasCommitLogLowerBound;
         }
 
         @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return hasCommitLogIntervals;
+        }
+
+        @Override
         public boolean storeRows()
         {
             return storeRows;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 4561520..a683513 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@ -24,6 +24,7 @@ import java.util.*;
 import com.google.common.collect.Maps;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
@@ -35,6 +36,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.StreamingHistogram;
 
+import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.replayPositionSetSerializer;
+
 /**
  * Serializer for SSTable from legacy versions
  */
@@ -55,7 +58,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
         EstimatedHistogram.serializer.serialize(stats.estimatedPartitionSize, out);
         EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-        ReplayPosition.serializer.serialize(stats.commitLogUpperBound, out);
+        ReplayPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
         out.writeLong(stats.minTimestamp);
         out.writeLong(stats.maxTimestamp);
         out.writeInt(stats.maxLocalDeletionTime);
@@ -72,7 +75,9 @@ public class LegacyMetadataSerializer extends MetadataSerializer
         for (ByteBuffer value : stats.maxClusteringValues)
             ByteBufferUtil.writeWithShortLength(value, out);
         if (version.hasCommitLogLowerBound())
-            ReplayPosition.serializer.serialize(stats.commitLogLowerBound, out);
+            ReplayPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+        if (version.hasCommitLogIntervals())
+            replayPositionSetSerializer.serialize(stats.commitLogIntervals, out);
     }
 
     /**
@@ -121,6 +126,11 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
                 if (descriptor.version.hasCommitLogLowerBound())
                     commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+                IntervalSet<ReplayPosition> commitLogIntervals;
+                if (descriptor.version.hasCommitLogIntervals())
+                    commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+                else
+                    commitLogIntervals = new IntervalSet<>(commitLogLowerBound, commitLogUpperBound);
 
                 if (types.contains(MetadataType.VALIDATION))
                     components.put(MetadataType.VALIDATION,
@@ -129,8 +139,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
                     components.put(MetadataType.STATS,
                                    new StatsMetadata(partitionSizes,
                                                      columnCounts,
-                                                     commitLogLowerBound,
-                                                     commitLogUpperBound,
+                                                     commitLogIntervals,
                                                      minTimestamp,
                                                      maxTimestamp,
                                                      Integer.MAX_VALUE,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 53cf0b0..1ff2ca8 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -27,12 +27,11 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
 
 import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 import com.clearspring.analytics.stream.cardinality.ICardinality;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
 import org.apache.cassandra.db.rows.Cell;
@@ -69,8 +68,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         return new StatsMetadata(defaultPartitionSizeHistogram(),
                                  defaultCellPerPartitionCountHistogram(),
-                                 ReplayPosition.NONE,
-                                 ReplayPosition.NONE,
+                                 IntervalSet.empty(),
                                  Long.MIN_VALUE,
                                  Long.MAX_VALUE,
                                  Integer.MAX_VALUE,
@@ -91,8 +89,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
     // TODO: cound the number of row per partition (either with the number of cells, or instead)
     protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
-    protected ReplayPosition commitLogLowerBound = ReplayPosition.NONE;
-    protected ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
+    protected IntervalSet commitLogIntervals = IntervalSet.empty();
     protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
     protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
     protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
@@ -126,23 +123,13 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         this(comparator);
 
-        ReplayPosition min = null, max = null;
+        IntervalSet.Builder intervals = new IntervalSet.Builder();
         for (SSTableReader sstable : sstables)
         {
-            if (min == null)
-            {
-                min = sstable.getSSTableMetadata().commitLogLowerBound;
-                max = sstable.getSSTableMetadata().commitLogUpperBound;
-            }
-            else
-            {
-                min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
-                max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
-            }
+            intervals.addAll(sstable.getSSTableMetadata().commitLogIntervals);
         }
 
-        commitLogLowerBound(min);
-        commitLogUpperBound(max);
+        commitLogIntervals(intervals.build());
         sstableLevel(level);
     }
 
@@ -229,15 +216,9 @@ public class MetadataCollector implements PartitionStatisticsCollector
         ttlTracker.update(newTTL);
     }
 
-    public MetadataCollector commitLogLowerBound(ReplayPosition commitLogLowerBound)
-    {
-        this.commitLogLowerBound = commitLogLowerBound;
-        return this;
-    }
-
-    public MetadataCollector commitLogUpperBound(ReplayPosition commitLogUpperBound)
+    public MetadataCollector commitLogIntervals(IntervalSet commitLogIntervals)
     {
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         return this;
     }
 
@@ -302,8 +283,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
         components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
         components.put(MetadataType.STATS, new StatsMetadata(estimatedPartitionSize,
                                                              estimatedCellPerPartitionCount,
-                                                             commitLogLowerBound,
-                                                             commitLogUpperBound,
+                                                             commitLogIntervals,
                                                              timestampTracker.min(),
                                                              timestampTracker.max(),
                                                              localDeletionTimeTracker.min(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index 07e35bb..9971eaa 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@ -22,10 +22,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -39,11 +41,11 @@ import org.apache.cassandra.utils.StreamingHistogram;
 public class StatsMetadata extends MetadataComponent
 {
     public static final IMetadataComponentSerializer serializer = new StatsMetadataSerializer();
+    public static final ISerializer<IntervalSet<ReplayPosition>> replayPositionSetSerializer = IntervalSet.serializer(ReplayPosition.serializer);
 
     public final EstimatedHistogram estimatedPartitionSize;
     public final EstimatedHistogram estimatedColumnCount;
-    public final ReplayPosition commitLogLowerBound;
-    public final ReplayPosition commitLogUpperBound;
+    public final IntervalSet<ReplayPosition> commitLogIntervals;
     public final long minTimestamp;
     public final long maxTimestamp;
     public final int minLocalDeletionTime;
@@ -62,8 +64,7 @@ public class StatsMetadata extends MetadataComponent
 
     public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                          EstimatedHistogram estimatedColumnCount,
-                         ReplayPosition commitLogLowerBound,
-                         ReplayPosition commitLogUpperBound,
+                         IntervalSet<ReplayPosition> commitLogIntervals,
                          long minTimestamp,
                          long maxTimestamp,
                          int minLocalDeletionTime,
@@ -82,8 +83,7 @@ public class StatsMetadata extends MetadataComponent
     {
         this.estimatedPartitionSize = estimatedPartitionSize;
         this.estimatedColumnCount = estimatedColumnCount;
-        this.commitLogLowerBound = commitLogLowerBound;
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         this.minTimestamp = minTimestamp;
         this.maxTimestamp = maxTimestamp;
         this.minLocalDeletionTime = minLocalDeletionTime;
@@ -134,8 +134,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -157,8 +156,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -186,8 +184,7 @@ public class StatsMetadata extends MetadataComponent
         return new EqualsBuilder()
                        .append(estimatedPartitionSize, that.estimatedPartitionSize)
                        .append(estimatedColumnCount, that.estimatedColumnCount)
-                       .append(commitLogLowerBound, that.commitLogLowerBound)
-                       .append(commitLogUpperBound, that.commitLogUpperBound)
+                       .append(commitLogIntervals, that.commitLogIntervals)
                        .append(minTimestamp, that.minTimestamp)
                        .append(maxTimestamp, that.maxTimestamp)
                        .append(minLocalDeletionTime, that.minLocalDeletionTime)
@@ -212,8 +209,7 @@ public class StatsMetadata extends MetadataComponent
         return new HashCodeBuilder()
                        .append(estimatedPartitionSize)
                        .append(estimatedColumnCount)
-                       .append(commitLogLowerBound)
-                       .append(commitLogUpperBound)
+                       .append(commitLogIntervals)
                        .append(minTimestamp)
                        .append(maxTimestamp)
                        .append(minLocalDeletionTime)
@@ -239,7 +235,7 @@ public class StatsMetadata extends MetadataComponent
             int size = 0;
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize);
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount);
-            size += ReplayPosition.serializer.serializedSize(component.commitLogUpperBound);
+            size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE));
             if (version.storeRows())
                 size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long)
             else
@@ -258,7 +254,9 @@ public class StatsMetadata extends MetadataComponent
             if (version.storeRows())
                 size += 8 + 8; // totalColumnsSet, totalRows
             if (version.hasCommitLogLowerBound())
-                size += ReplayPosition.serializer.serializedSize(component.commitLogLowerBound);
+                size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE));
+            if (version.hasCommitLogIntervals())
+                size += replayPositionSetSerializer.serializedSize(component.commitLogIntervals);
             return size;
         }
 
@@ -266,7 +264,7 @@ public class StatsMetadata extends MetadataComponent
         {
             EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out);
             EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-            ReplayPosition.serializer.serialize(component.commitLogUpperBound, out);
+            ReplayPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
             out.writeLong(component.minTimestamp);
             out.writeLong(component.maxTimestamp);
             if (version.storeRows())
@@ -296,7 +294,9 @@ public class StatsMetadata extends MetadataComponent
             }
 
             if (version.hasCommitLogLowerBound())
-                ReplayPosition.serializer.serialize(component.commitLogLowerBound, out);
+                ReplayPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+            if (version.hasCommitLogIntervals())
+                replayPositionSetSerializer.serialize(component.commitLogIntervals, out);
         }
 
         public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@ -338,11 +338,15 @@ public class StatsMetadata extends MetadataComponent
 
             if (version.hasCommitLogLowerBound())
                 commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+            IntervalSet<ReplayPosition> commitLogIntervals;
+            if (version.hasCommitLogIntervals())
+                commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+            else
+                commitLogIntervals = new IntervalSet<ReplayPosition>(commitLogLowerBound, commitLogUpperBound);
 
             return new StatsMetadata(partitionSizes,
                                      columnCounts,
-                                     commitLogLowerBound,
-                                     commitLogUpperBound,
+                                     commitLogIntervals,
                                      minTimestamp,
                                      maxTimestamp,
                                      minLocalDeletionTime,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 420b802..5f7513f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -70,8 +70,7 @@ public class SSTableMetadataViewer
                     out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                     out.printf("SSTable Level: %d%n", stats.sstableLevel);
                     out.printf("Repaired at: %d%n", stats.repairedAt);
-                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
-                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                     out.println("Estimated tombstone drop times:");
                     for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                     {


[20/23] cassandra git commit: Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

Posted by sl...@apache.org.
Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

* commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0':
  Change commitlog and sstables to track dirty and clean intervals.
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/trunk
Commit: 7b1021733b55c8865f80e261697b4c079d086633
Parents: 21c92ca 904cb5d
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:39:15 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:39:56 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  70 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  21 +-
 .../AbstractCommitLogSegmentManager.java        |   4 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         | 105 ++----
 .../db/commitlog/CommitLogSegment.java          |  82 +++--
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  45 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  37 +--
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  12 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  37 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   3 +-
 test/unit/org/apache/cassandra/Util.java        |  21 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 151 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 97 files changed, 1222 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 289f370,b596fc9..43d28f3
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,5 +1,11 @@@
 -3.0.9
 +3.9
 + * Fix nodetool tablestats miss SSTable count (CASSANDRA-12205)
 + * Fixed flacky SSTablesIteratedTest (CASSANDRA-12282)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * cqlsh: Fix handling of $$-escaped strings (CASSANDRA-12189)
 + * Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
 +Merged from 3.0:
+  * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
   * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
   * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
   * Fix upgrade of super columns on thrift (CASSANDRA-12335)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 9d31b60,82604e2..21becfe
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -957,29 -922,19 +954,20 @@@ public class ColumnFamilyStore implemen
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
-         final CommitLogPosition commitLogUpperBound;
 -        volatile FSWriteError flushFailure = null;
 +        volatile Throwable flushFailure = null;
          final List<Memtable> memtables;
-         final List<Collection<SSTableReader>> readers;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier,
 +        private PostFlush(boolean flushSecondaryIndexes,
 +                          OpOrder.Barrier writeBarrier,
-                           CommitLogPosition commitLogUpperBound,
-                           List<Memtable> memtables,
-                           List<Collection<SSTableReader>> readers)
+                           List<Memtable> memtables)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
-             this.commitLogUpperBound = commitLogUpperBound;
              this.memtables = memtables;
-             this.readers = readers;
          }
  
 -        public ReplayPosition call()
 +        public CommitLogPosition call()
          {
-             if (discardFlushResults == ColumnFamilyStore.this)
-                 return commitLogUpperBound;
- 
              writeBarrier.await();
  
              /**
@@@ -1003,19 -958,13 +991,13 @@@
                  throw new IllegalStateException();
              }
  
-             // Must check commitLogUpperBound != null because Flush may find that all memtables are clean
-             // and so not set a commitLogUpperBound
 -            ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
++            CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE;
              // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
+             if (flushFailure == null && !memtables.isEmpty())
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     Collection<SSTableReader> reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, reader);
-                 }
+                 Memtable memtable = memtables.get(0);
+                 commitLogUpperBound = memtable.getCommitLogUpperBound();
+                 CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
              }
  
              metric.pendingFlushes.dec();
@@@ -1079,9 -1027,9 +1060,9 @@@
  
              // we then issue the barrier; this lets us wait for all operations started prior to the barrier to complete;
              // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
 -            // replay positions have also completed, i.e. the memtables are done and ready to flush
 +            // commit log segment position have also completed, i.e. the memtables are done and ready to flush
              writeBarrier.issue();
-             postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
+             postFlush = new PostFlush(!truncate, writeBarrier, memtables);
          }
  
          public void run()
@@@ -1111,110 -1059,23 +1092,108 @@@
              try
              {
                  for (Memtable memtable : memtables)
--                {
-                     this.readers.add(flushMemtable(memtable));
 -                    Collection<SSTableReader> readers = memtable.flush();
 -                    memtable.cfs.replaceFlushed(memtable, readers);
 -                    reclaim(memtable);
--                }
++                    flushMemtable(memtable);
              }
 -            catch (FSWriteError e)
 +            catch (Throwable t)
              {
 -                JVMStabilityInspector.inspectThrowable(e);
 -                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
 -                postFlush.flushFailure = e;
 +                JVMStabilityInspector.inspectThrowable(t);
 +                postFlush.flushFailure = t;
              }
 -
              // signal the post-flush we've done our work
              postFlush.latch.countDown();
          }
  
 +        public Collection<SSTableReader> flushMemtable(Memtable memtable)
 +        {
 +            List<Future<SSTableMultiWriter>> futures = new ArrayList<>();
 +            long totalBytesOnDisk = 0;
 +            long maxBytesOnDisk = 0;
 +            long minBytesOnDisk = Long.MAX_VALUE;
 +            List<SSTableReader> sstables = new ArrayList<>();
 +            try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.FLUSH))
 +            {
 +                List<Memtable.FlushRunnable> flushRunnables = null;
 +                List<SSTableMultiWriter> flushResults = null;
 +
 +                try
 +                {
 +                    // flush the memtable
 +                    flushRunnables = memtable.flushRunnables(txn);
 +
 +                    for (int i = 0; i < flushRunnables.size(); i++)
 +                        futures.add(perDiskflushExecutors[i].submit(flushRunnables.get(i)));
 +
 +                    flushResults = Lists.newArrayList(FBUtilities.waitOnFutures(futures));
 +                }
 +                catch (Throwable t)
 +                {
 +                    t = memtable.abortRunnables(flushRunnables, t);
 +                    t = txn.abort(t);
 +                    throw Throwables.propagate(t);
 +                }
 +
 +                try
 +                {
 +                    Iterator<SSTableMultiWriter> writerIterator = flushResults.iterator();
 +                    while (writerIterator.hasNext())
 +                    {
 +                        @SuppressWarnings("resource")
 +                        SSTableMultiWriter writer = writerIterator.next();
 +                        if (writer.getFilePointer() > 0)
 +                        {
 +                            writer.setOpenResult(true).prepareToCommit();
 +                        }
 +                        else
 +                        {
 +                            maybeFail(writer.abort(null));
 +                            writerIterator.remove();
 +                        }
 +                    }
 +                }
 +                catch (Throwable t)
 +                {
 +                    for (SSTableMultiWriter writer : flushResults)
 +                        t = writer.abort(t);
 +                    t = txn.abort(t);
 +                    Throwables.propagate(t);
 +                }
 +
 +                txn.prepareToCommit();
 +
 +                Throwable accumulate = null;
 +                for (SSTableMultiWriter writer : flushResults)
 +                    accumulate = writer.commit(accumulate);
 +
 +                maybeFail(txn.commit(accumulate));
 +
 +                for (SSTableMultiWriter writer : flushResults)
 +                {
 +                    Collection<SSTableReader> flushedSSTables = writer.finished();
 +                    for (SSTableReader sstable : flushedSSTables)
 +                    {
 +                        if (sstable != null)
 +                        {
 +                            sstables.add(sstable);
 +                            long size = sstable.bytesOnDisk();
 +                            totalBytesOnDisk += size;
 +                            maxBytesOnDisk = Math.max(maxBytesOnDisk, size);
 +                            minBytesOnDisk = Math.min(minBytesOnDisk, size);
 +                        }
 +                    }
 +                }
 +            }
-             memtable.cfs.data.replaceFlushed(memtable, sstables);
++            memtable.cfs.replaceFlushed(memtable, sstables);
 +            reclaim(memtable);
 +            memtable.cfs.compactionStrategyManager.compactionLogger.flush(sstables);
 +            logger.debug("Flushed to {} ({} sstables, {}), biggest {}, smallest {}",
 +                         sstables,
 +                         sstables.size(),
 +                         FBUtilities.prettyPrintMemory(totalBytesOnDisk),
 +                         FBUtilities.prettyPrintMemory(maxBytesOnDisk),
 +                         FBUtilities.prettyPrintMemory(minBytesOnDisk));
 +            return sstables;
 +        }
 +
          private void reclaim(final Memtable memtable)
          {
              // issue a read barrier for reclaiming the memory, and offload the wait to another thread
@@@ -2268,10 -2085,10 +2222,10 @@@
      {
          Callable<LifecycleTransaction> callable = new Callable<LifecycleTransaction>()
          {
 -            public LifecycleTransaction call() throws Exception
 +            public LifecycleTransaction call()
              {
                  assert data.getCompacting().isEmpty() : data.getCompacting();
-                 Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                 Iterable<SSTableReader> sstables = getLiveSSTables();
                  sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
                  sstables = ImmutableList.copyOf(sstables);
                  LifecycleTransaction modifier = data.tryModify(sstables, operationType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Memtable.java
index 7a46d8a,3c77092..e9cca4a
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@@ -33,7 -33,9 +33,9 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.commitlog.CommitLog;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
+ import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.db.filter.ClusteringIndexFilter;
  import org.apache.cassandra.db.filter.ColumnFilter;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@@ -192,6 -194,11 +194,11 @@@ public class Memtable implements Compar
          return commitLogLowerBound.get();
      }
  
 -    public ReplayPosition getCommitLogUpperBound()
++    public CommitLogPosition getCommitLogUpperBound()
+     {
+         return commitLogUpperBound.get();
+     }
+ 
      public boolean isLive()
      {
          return allocator.isLive();
@@@ -361,63 -337,39 +368,72 @@@
          return minTimestamp;
      }
  
+     /**
+      * For testing only. Give this memtable too big a size to make it always fail flushing.
+      */
+     @VisibleForTesting
+     public void makeUnflushable()
+     {
+         liveDataSize.addAndGet(1L * 1024 * 1024 * 1024 * 1024 * 1024);
+     }
+ 
 -    private long estimatedSize()
 +    class FlushRunnable implements Callable<SSTableMultiWriter>
      {
 -        long keySize = 0;
 -        for (PartitionPosition key : partitions.keySet())
 +        private final long estimatedSize;
 +        private final ConcurrentNavigableMap<PartitionPosition, AtomicBTreePartition> toFlush;
 +
 +        private final boolean isBatchLogTable;
 +        private final SSTableMultiWriter writer;
 +
 +        // keeping these to be able to log what we are actually flushing
 +        private final PartitionPosition from;
 +        private final PartitionPosition to;
 +
 +        FlushRunnable(PartitionPosition from, PartitionPosition to, Directories.DataDirectory flushLocation, LifecycleTransaction txn)
          {
 -            //  make sure we don't write non-sensical keys
 -            assert key instanceof DecoratedKey;
 -            keySize += ((DecoratedKey)key).getKey().remaining();
 +            this(partitions.subMap(from, to), flushLocation, from, to, txn);
          }
 -        return (long) ((keySize // index entries
 -                        + keySize // keys in data file
 -                        + liveDataSize.get()) // data
 -                       * 1.2); // bloom filter and row index overhead
 -    }
  
 -    private Collection<SSTableReader> writeSortedContents(File sstableDirectory)
 -    {
 -        boolean isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SystemKeyspace.NAME);
 +        FlushRunnable(LifecycleTransaction txn)
 +        {
 +            this(partitions, null, null, null, txn);
 +        }
 +
 +        FlushRunnable(ConcurrentNavigableMap<PartitionPosition, AtomicBTreePartition> toFlush, Directories.DataDirectory flushLocation, PartitionPosition from, PartitionPosition to, LifecycleTransaction txn)
 +        {
 +            this.toFlush = toFlush;
 +            this.from = from;
 +            this.to = to;
 +            long keySize = 0;
 +            for (PartitionPosition key : toFlush.keySet())
 +            {
 +                //  make sure we don't write non-sensical keys
 +                assert key instanceof DecoratedKey;
 +                keySize += ((DecoratedKey) key).getKey().remaining();
 +            }
 +            estimatedSize = (long) ((keySize // index entries
 +                                    + keySize // keys in data file
 +                                    + liveDataSize.get()) // data
 +                                    * 1.2); // bloom filter and row index overhead
 +
 +            this.isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SystemKeyspace.NAME);
  
 -        logger.debug("Writing {}", Memtable.this.toString());
 +            if (flushLocation == null)
 +                writer = createFlushWriter(txn, cfs.getSSTablePath(getDirectories().getWriteableLocationAsFile(estimatedSize)), columnsCollector.get(), statsCollector.get());
 +            else
 +                writer = createFlushWriter(txn, cfs.getSSTablePath(getDirectories().getLocationForDisk(flushLocation)), columnsCollector.get(), statsCollector.get());
 +
 +        }
  
 -        Collection<SSTableReader> ssTables;
 -        try (SSTableTxnWriter writer = createFlushWriter(cfs.getSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get()))
 +        protected Directories getDirectories()
          {
 +            return cfs.getDirectories();
 +        }
 +
 +        private void writeSortedContents()
 +        {
 +            logger.debug("Writing {}, flushed range = ({}, {}]", Memtable.this.toString(), from, to);
 +
              boolean trackContention = logger.isTraceEnabled();
              int heavilyContendedRowCount = 0;
              // (we can't clear out the map as-we-go to free up memory,
@@@ -444,39 -396,58 +460,38 @@@
                  }
              }
  
 -            if (writer.getFilePointer() > 0)
 -            {
 -                logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
 -                                           writer.getFilename(),
 -                                           FBUtilities.prettyPrintMemory(writer.getFilePointer()),
 -                                           commitLogUpperBound));
 -
 -                // sstables should contain non-repaired data.
 -                ssTables = writer.finish(true);
 -            }
 -            else
 -            {
 -                logger.debug("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
 -                             writer.getFilename(), commitLogUpperBound);
 -                writer.abort();
 -                ssTables = Collections.emptyList();
 -            }
 +            long bytesFlushed = writer.getFilePointer();
 +            logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
 +                                                                              writer.getFilename(),
 +                                                                              FBUtilities.prettyPrintMemory(bytesFlushed),
 +                                                                              commitLogUpperBound));
 +            // Update the metrics
 +            cfs.metric.bytesFlushed.inc(bytesFlushed);
  
              if (heavilyContendedRowCount > 0)
 -                logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, partitions.size(), Memtable.this.toString()));
 -
 -            return ssTables;
 +                logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, toFlush.size(), Memtable.this.toString()));
          }
 -    }
  
 -    @SuppressWarnings("resource") // log and writer closed by SSTableTxnWriter
 -    public SSTableTxnWriter createFlushWriter(String filename,
 -                                              PartitionColumns columns,
 -                                              EncodingStats stats)
 -    {
 -        // we operate "offline" here, as we expose the resulting reader consciously when done
 -        // (although we may want to modify this behaviour in future, to encapsulate full flush behaviour in LifecycleTransaction)
 -        LifecycleTransaction txn = null;
 -        try
 +        public SSTableMultiWriter createFlushWriter(LifecycleTransaction txn,
 +                                                  String filename,
 +                                                  PartitionColumns columns,
 +                                                  EncodingStats stats)
          {
 -            txn = LifecycleTransaction.offline(OperationType.FLUSH);
              MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
-                     .commitLogLowerBound(commitLogLowerBound.get())
-                     .commitLogUpperBound(commitLogUpperBound.get());
 -                    .commitLogIntervals(new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));
 -
 -            return new SSTableTxnWriter(txn,
 -                                        cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),
 -                                                                     (long) partitions.size(),
 -                                                                     ActiveRepairService.UNREPAIRED_SSTABLE,
 -                                                                     sstableMetadataCollector,
 -                                                                     new SerializationHeader(true, cfs.metadata, columns, stats),
 -                                                                     txn));
++                    .commitLogIntervals(new IntervalSet<>(commitLogLowerBound.get(), commitLogUpperBound.get()));
++
 +            return cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),
 +                                                (long)toFlush.size(),
 +                                                ActiveRepairService.UNREPAIRED_SSTABLE,
 +                                                sstableMetadataCollector,
 +                                                new SerializationHeader(true, cfs.metadata, columns, stats), txn);
- 
          }
 -        catch (Throwable t)
 +
 +        @Override
 +        public SSTableMultiWriter call()
          {
 -            if (txn != null)
 -                txn.close();
 -            throw t;
 +            writeSortedContents();
 +            return writer;
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index 7ea7439,0000000..8f3b7e4
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@@ -1,582 -1,0 +1,582 @@@
 +/*
 + * 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.commitlog;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicLong;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.collect.Iterables;
 +import com.google.common.util.concurrent.*;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.WaitQueue;
 +
 +import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
 +
 +/**
 + * Performs eager-creation of commit log segments in a background thread. All the
 + * public methods are thread safe.
 + */
 +public abstract class AbstractCommitLogSegmentManager
 +{
 +    static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogSegmentManager.class);
 +
 +    // Queue of work to be done by the manager thread, also used to wake the thread to perform segment allocation.
 +    private final BlockingQueue<Runnable> segmentManagementTasks = new LinkedBlockingQueue<>();
 +
 +    /** Segments that are ready to be used. Head of the queue is the one we allocate writes to */
 +    private final ConcurrentLinkedQueue<CommitLogSegment> availableSegments = new ConcurrentLinkedQueue<>();
 +
 +    /** Active segments, containing unflushed data */
 +    private final ConcurrentLinkedQueue<CommitLogSegment> activeSegments = new ConcurrentLinkedQueue<>();
 +
 +    /** The segment we are currently allocating commit log records to */
 +    protected volatile CommitLogSegment allocatingFrom = null;
 +
 +    private final WaitQueue hasAvailableSegments = new WaitQueue();
 +
 +    final String storageDirectory;
 +
 +    /**
 +     * Tracks commitlog size, in multiples of the segment size.  We need to do this so we can "promise" size
 +     * adjustments ahead of actually adding/freeing segments on disk, so that the "evict oldest segment" logic
 +     * can see the effect of recycling segments immediately (even though they're really happening asynchronously
 +     * on the manager thread, which will take a ms or two).
 +     */
 +    private final AtomicLong size = new AtomicLong();
 +
 +    /**
 +     * New segment creation is initially disabled because we'll typically get some "free" segments
 +     * recycled after log replay.
 +     */
 +    volatile boolean createReserveSegments = false;
 +
 +    private Thread managerThread;
 +    protected volatile boolean run = true;
 +    protected final CommitLog commitLog;
 +
 +    private static final SimpleCachedBufferPool bufferPool =
 +        new SimpleCachedBufferPool(DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool(), DatabaseDescriptor.getCommitLogSegmentSize());
 +
 +    AbstractCommitLogSegmentManager(final CommitLog commitLog, String storageDirectory)
 +    {
 +        this.commitLog = commitLog;
 +        this.storageDirectory = storageDirectory;
 +    }
 +
 +    void start()
 +    {
 +        // The run loop for the manager thread
 +        Runnable runnable = new WrappedRunnable()
 +        {
 +            public void runMayThrow() throws Exception
 +            {
 +                while (run)
 +                {
 +                    try
 +                    {
 +                        Runnable task = segmentManagementTasks.poll();
 +                        if (task == null)
 +                        {
 +                            // if we have no more work to do, check if we should create a new segment
 +                            if (!atSegmentLimit() &&
 +                                availableSegments.isEmpty() &&
 +                                (activeSegments.isEmpty() || createReserveSegments))
 +                            {
 +                                logger.trace("No segments in reserve; creating a fresh one");
 +                                // TODO : some error handling in case we fail to create a new segment
 +                                availableSegments.add(createSegment());
 +                                hasAvailableSegments.signalAll();
 +                            }
 +
 +                            // flush old Cfs if we're full
 +                            long unused = unusedCapacity();
 +                            if (unused < 0)
 +                            {
 +                                List<CommitLogSegment> segmentsToRecycle = new ArrayList<>();
 +                                long spaceToReclaim = 0;
 +                                for (CommitLogSegment segment : activeSegments)
 +                                {
 +                                    if (segment == allocatingFrom)
 +                                        break;
 +                                    segmentsToRecycle.add(segment);
 +                                    spaceToReclaim += DatabaseDescriptor.getCommitLogSegmentSize();
 +                                    if (spaceToReclaim + unused >= 0)
 +                                        break;
 +                                }
 +                                flushDataFrom(segmentsToRecycle, false);
 +                            }
 +
 +                            // Since we're operating on a "null" allocation task, block here for the next task on the
 +                            // queue rather than looping, grabbing another null, and repeating the above work.
 +                            try
 +                            {
 +                                task = segmentManagementTasks.take();
 +                            }
 +                            catch (InterruptedException e)
 +                            {
 +                                throw new AssertionError();
 +                            }
 +                        }
 +                        task.run();
 +                    }
 +                    catch (Throwable t)
 +                    {
 +                        JVMStabilityInspector.inspectThrowable(t);
 +                        if (!CommitLog.handleCommitError("Failed managing commit log segments", t))
 +                            return;
 +                        // sleep some arbitrary period to avoid spamming CL
 +                        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 +                    }
 +                }
 +            }
 +
 +            private boolean atSegmentLimit()
 +            {
 +                return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
 +            }
 +        };
 +
 +        run = true;
 +
 +        managerThread = new Thread(runnable, "COMMIT-LOG-ALLOCATOR");
 +        managerThread.start();
 +    }
 +
 +
 +    /**
 +     * Shut down the CLSM. Used both during testing and during regular shutdown, so needs to stop everything.
 +     */
 +    public abstract void shutdown();
 +
 +    /**
 +     * Allocate a segment within this CLSM. Should either succeed or throw.
 +     */
 +    public abstract Allocation allocate(Mutation mutation, int size);
 +
 +    /**
 +     * The recovery and replay process replays mutations into memtables and flushes them to disk. Individual CLSM
 +     * decide what to do with those segments on disk after they've been replayed.
 +     */
 +    abstract void handleReplayedSegment(final File file);
 +
 +    /**
 +     * Hook to allow segment managers to track state surrounding creation of new segments. Onl perform as task submit
 +     * to segment manager so it's performed on segment management thread.
 +     */
 +    abstract CommitLogSegment createSegment();
 +
 +    /**
 +     * Indicates that a segment file has been flushed and is no longer needed. Only perform as task submit to segment
 +     * manager so it's performend on segment management thread, or perform while segment management thread is shutdown
 +     * during testing resets.
 +     *
 +     * @param segment segment to be discarded
 +     * @param delete  whether or not the segment is safe to be deleted.
 +     */
 +    abstract void discard(CommitLogSegment segment, boolean delete);
 +
 +
 +    /**
 +     * Grab the current CommitLogSegment we're allocating from. Also serves as a utility method to block while the allocator
 +     * is working on initial allocation of a CommitLogSegment.
 +     */
 +    CommitLogSegment allocatingFrom()
 +    {
 +        CommitLogSegment r = allocatingFrom;
 +        if (r == null)
 +        {
 +            advanceAllocatingFrom(null);
 +            r = allocatingFrom;
 +        }
 +        return r;
 +    }
 +
 +    /**
 +     * Fetches a new segment from the queue, signaling the management thread to create a new one if necessary, and "activates" it.
 +     * Blocks until a new segment is allocated and the thread requesting an advanceAllocatingFrom is signalled.
 +     *
 +     * WARNING: Assumes segment management thread always succeeds in allocating a new segment or kills the JVM.
 +     */
 +    protected void advanceAllocatingFrom(CommitLogSegment old)
 +    {
 +        while (true)
 +        {
 +            CommitLogSegment next;
 +            synchronized (this)
 +            {
 +                // do this in a critical section so we can atomically remove from availableSegments and add to allocatingFrom/activeSegments
 +                // see https://issues.apache.org/jira/browse/CASSANDRA-6557?focusedCommentId=13874432&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13874432
 +                if (allocatingFrom != old)
 +                    return;
 +                next = availableSegments.poll();
 +                if (next != null)
 +                {
 +                    allocatingFrom = next;
 +                    activeSegments.add(next);
 +                }
 +            }
 +
 +            if (next != null)
 +            {
 +                if (old != null)
 +                {
 +                    // Now we can run the user defined command just after switching to the new commit log.
 +                    // (Do this here instead of in the recycle call so we can get a head start on the archive.)
 +                    commitLog.archiver.maybeArchive(old);
 +
 +                    // ensure we don't continue to use the old file; not strictly necessary, but cleaner to enforce it
 +                    old.discardUnusedTail();
 +                }
 +
 +                // request that the CL be synced out-of-band, as we've finished a segment
 +                commitLog.requestExtraSync();
 +                return;
 +            }
 +
 +            // no more segments, so register to receive a signal when not empty
 +            WaitQueue.Signal signal = hasAvailableSegments.register(commitLog.metrics.waitingOnSegmentAllocation.time());
 +
 +            // trigger the management thread; this must occur after registering
 +            // the signal to ensure we are woken by any new segment creation
 +            wakeManager();
 +
 +            // check if the queue has already been added to before waiting on the signal, to catch modifications
 +            // that happened prior to registering the signal; *then* check to see if we've been beaten to making the change
 +            if (!availableSegments.isEmpty() || allocatingFrom != old)
 +            {
 +                signal.cancel();
 +                // if we've been beaten, just stop immediately
 +                if (allocatingFrom != old)
 +                    return;
 +                // otherwise try again, as there should be an available segment
 +                continue;
 +            }
 +
 +            // can only reach here if the queue hasn't been inserted into
 +            // before we registered the signal, as we only remove items from the queue
 +            // after updating allocatingFrom. Can safely block until we are signalled
 +            // by the allocator that new segments have been published
 +            signal.awaitUninterruptibly();
 +        }
 +    }
 +
 +    protected void wakeManager()
 +    {
 +        // put a NO-OP on the queue, to trigger management thread (and create a new segment if necessary)
 +        segmentManagementTasks.add(Runnables.doNothing());
 +    }
 +
 +    /**
 +     * Switch to a new segment, regardless of how much is left in the current one.
 +     *
 +     * Flushes any dirty CFs for this segment and any older segments, and then recycles
 +     * the segments
 +     */
 +    void forceRecycleAll(Iterable<UUID> droppedCfs)
 +    {
 +        List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
 +        CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1);
 +        advanceAllocatingFrom(last);
 +
 +        // wait for the commit log modifications
 +        last.waitForModifications();
 +
 +        // make sure the writes have materialized inside of the memtables by waiting for all outstanding writes
 +        // on the relevant keyspaces to complete
 +        Keyspace.writeOrder.awaitNewBarrier();
 +
 +        // flush and wait for all CFs that are dirty in segments up-to and including 'last'
 +        Future<?> future = flushDataFrom(segmentsToRecycle, true);
 +        try
 +        {
 +            future.get();
 +
 +            for (CommitLogSegment segment : activeSegments)
 +                for (UUID cfId : droppedCfs)
-                     segment.markClean(cfId, segment.getCurrentCommitLogPosition());
++                    segment.markClean(cfId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
 +
 +            // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
 +            // if the previous active segment was the only one to recycle (since an active segment isn't
 +            // necessarily dirty, and we only call dCS after a flush).
 +            for (CommitLogSegment segment : activeSegments)
 +            {
 +                if (segment.isUnused())
 +                    recycleSegment(segment);
 +            }
 +
 +            CommitLogSegment first;
 +            if ((first = activeSegments.peek()) != null && first.id <= last.id)
 +                logger.error("Failed to force-recycle all segments; at least one segment is still in use with dirty CFs.");
 +        }
 +        catch (Throwable t)
 +        {
 +            // for now just log the error
 +            logger.error("Failed waiting for a forced recycle of in-use commit log segments", t);
 +        }
 +    }
 +
 +    /**
 +     * Indicates that a segment is no longer in use and that it should be recycled.
 +     *
 +     * @param segment segment that is no longer in use
 +     */
 +    void recycleSegment(final CommitLogSegment segment)
 +    {
 +        boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName());
 +        if (activeSegments.remove(segment))
 +        {
 +            // if archiving (command) was not successful then leave the file alone. don't delete or recycle.
 +            discardSegment(segment, archiveSuccess);
 +        }
 +        else
 +        {
 +            logger.warn("segment {} not found in activeSegments queue", segment);
 +        }
 +    }
 +
 +    /**
 +     * Indicates that a segment file should be deleted.
 +     *
 +     * @param segment segment to be discarded
 +     */
 +    private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
 +    {
 +        logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
 +        segmentManagementTasks.add(() -> discard(segment, deleteFile));
 +    }
 +
 +    /**
 +     * Adjust the tracked on-disk size. Called by individual segments to reflect writes, allocations and discards.
 +     * @param addedSize
 +     */
 +    void addSize(long addedSize)
 +    {
 +        size.addAndGet(addedSize);
 +    }
 +
 +    /**
 +     * @return the space (in bytes) used by all segment files.
 +     */
 +    public long onDiskSize()
 +    {
 +        return size.get();
 +    }
 +
 +    private long unusedCapacity()
 +    {
 +        long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
 +        long currentSize = size.get();
 +        logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total);
 +        return total - currentSize;
 +    }
 +
 +    /**
 +     * @param name the filename to check
 +     * @return true if file is managed by this manager.
 +     */
 +    public boolean manages(String name)
 +    {
 +        for (CommitLogSegment segment : Iterables.concat(activeSegments, availableSegments))
 +            if (segment.getName().equals(name))
 +                return true;
 +        return false;
 +    }
 +
 +    /**
 +     * Throws a flag that enables the behavior of keeping at least one spare segment
 +     * available at all times.
 +     */
 +    void enableReserveSegmentCreation()
 +    {
 +        createReserveSegments = true;
 +        wakeManager();
 +    }
 +
 +    /**
 +     * Force a flush on all CFs that are still dirty in @param segments.
 +     *
 +     * @return a Future that will finish when all the flushes are complete.
 +     */
 +    private Future<?> flushDataFrom(List<CommitLogSegment> segments, boolean force)
 +    {
 +        if (segments.isEmpty())
 +            return Futures.immediateFuture(null);
 +        final CommitLogPosition maxCommitLogPosition = segments.get(segments.size() - 1).getCurrentCommitLogPosition();
 +
 +        // a map of CfId -> forceFlush() to ensure we only queue one flush per cf
 +        final Map<UUID, ListenableFuture<?>> flushes = new LinkedHashMap<>();
 +
 +        for (CommitLogSegment segment : segments)
 +        {
 +            for (UUID dirtyCFId : segment.getDirtyCFIDs())
 +            {
 +                Pair<String,String> pair = Schema.instance.getCF(dirtyCFId);
 +                if (pair == null)
 +                {
 +                    // even though we remove the schema entry before a final flush when dropping a CF,
 +                    // it's still possible for a writer to race and finish his append after the flush.
 +                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                     segment.markClean(dirtyCFId, segment.getCurrentCommitLogPosition());
++                    segment.markClean(dirtyCFId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
 +                }
 +                else if (!flushes.containsKey(dirtyCFId))
 +                {
 +                    String keyspace = pair.left;
 +                    final ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(dirtyCFId);
 +                    // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush,
 +                    // no deadlock possibility since switchLock removal
 +                    flushes.put(dirtyCFId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition));
 +                }
 +            }
 +        }
 +
 +        return Futures.allAsList(flushes.values());
 +    }
 +
 +    /**
 +     * Stops CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
 +     * Only call this after the AbstractCommitLogService is shut down.
 +     */
 +    public void stopUnsafe(boolean deleteSegments)
 +    {
 +        logger.trace("CLSM closing and clearing existing commit log segments...");
 +        createReserveSegments = false;
 +
 +        awaitManagementTasksCompletion();
 +
 +        shutdown();
 +        try
 +        {
 +            awaitTermination();
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        synchronized (this)
 +        {
 +            for (CommitLogSegment segment : activeSegments)
 +                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
 +            activeSegments.clear();
 +
 +            for (CommitLogSegment segment : availableSegments)
 +                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
 +            availableSegments.clear();
 +        }
 +
 +        allocatingFrom = null;
 +
 +        segmentManagementTasks.clear();
 +
 +        size.set(0L);
 +
 +        logger.trace("CLSM done with closing and clearing existing commit log segments.");
 +    }
 +
 +    // Used by tests only.
 +    void awaitManagementTasksCompletion()
 +    {
 +        while (!segmentManagementTasks.isEmpty())
 +            Thread.yield();
 +        // The last management task is not yet complete. Wait a while for it.
 +        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +        // TODO: If this functionality is required by anything other than tests, signalling must be used to ensure
 +        // waiting completes correctly.
 +    }
 +
 +    /**
 +     * Explicitly for use only during resets in unit testing.
 +     */
 +    private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delete)
 +    {
 +        try
 +        {
 +            discard(segment, delete);
 +        }
 +        catch (AssertionError ignored)
 +        {
 +            // segment file does not exist
 +        }
 +    }
 +
 +    /**
 +     * Returns when the management thread terminates.
 +     */
 +    public void awaitTermination() throws InterruptedException
 +    {
 +        managerThread.join();
 +
 +        for (CommitLogSegment segment : activeSegments)
 +            segment.close();
 +
 +        for (CommitLogSegment segment : availableSegments)
 +            segment.close();
 +
 +        bufferPool.shutdown();
 +    }
 +
 +    /**
 +     * @return a read-only collection of the active commit log segments
 +     */
 +    @VisibleForTesting
 +    public Collection<CommitLogSegment> getActiveSegments()
 +    {
 +        return Collections.unmodifiableCollection(activeSegments);
 +    }
 +
 +    /**
 +     * @return the current CommitLogPosition of the active segment we're allocating from
 +     */
 +    CommitLogPosition getCurrentPosition()
 +    {
 +        return allocatingFrom().getCurrentCommitLogPosition();
 +    }
 +
 +    /**
 +     * Forces a disk flush on the commit log files that need it.  Blocking.
 +     */
 +    public void sync(boolean syncAllSegments) throws IOException
 +    {
 +        CommitLogSegment current = allocatingFrom();
 +        for (CommitLogSegment segment : getActiveSegments())
 +        {
 +            if (!syncAllSegments && segment.id > current.id)
 +                return;
 +            segment.sync();
 +        }
 +    }
 +
 +    /**
 +     * Used by compressed and encrypted segments to share a buffer pool across the CLSM.
 +     */
 +    SimpleCachedBufferPool getBufferPool()
 +    {
 +        return bufferPool;
 +    }
 +}
 +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index b66221c,dfe3f91..32f69eb
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@@ -298,20 -290,21 +298,21 @@@ public class CommitLog implements Commi
       * given. Discards any commit log segments that are no longer used.
       *
       * @param cfId    the column family ID that was flushed
-      * @param context the commit log segment position of the flush
+      * @param lowerBound the lowest covered replay position of the flush
+      * @param lowerBound the highest covered replay position of the flush
       */
-     public void discardCompletedSegments(final UUID cfId, final CommitLogPosition context)
 -    public void discardCompletedSegments(final UUID cfId, final ReplayPosition lowerBound, final ReplayPosition upperBound)
++    public void discardCompletedSegments(final UUID cfId, final CommitLogPosition lowerBound, final CommitLogPosition upperBound)
      {
-         logger.trace("discard completed log segments for {}, table {}", context, cfId);
+         logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
  
          // Go thru the active segment files, which are ordered oldest to newest, marking the
 -        // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
 +        // flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed
          // in the arguments. Any segments that become unused after they are marked clean will be
          // recycled or discarded.
 -        for (Iterator<CommitLogSegment> iter = allocator.getActiveSegments().iterator(); iter.hasNext();)
 +        for (Iterator<CommitLogSegment> iter = segmentManager.getActiveSegments().iterator(); iter.hasNext();)
          {
              CommitLogSegment segment = iter.next();
-             segment.markClean(cfId, context);
+             segment.markClean(cfId, lowerBound, upperBound);
  
              if (segment.isUnused())
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index c8e597f,af8efb4..92364c8
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@@ -54,32 -70,31 +54,32 @@@ public class CommitLogReplayer implemen
      static final String IGNORE_REPLAY_ERRORS_PROPERTY = "cassandra.commitlog.ignorereplayerrors";
      private static final Logger logger = LoggerFactory.getLogger(CommitLogReplayer.class);
      private static final int MAX_OUTSTANDING_REPLAY_COUNT = Integer.getInteger("cassandra.commitlog_max_outstanding_replay_count", 1024);
 -    private static final int LEGACY_END_OF_SEGMENT_MARKER = 0;
  
 -    private final Set<Keyspace> keyspacesRecovered;
 -    private final List<Future<?>> futures;
 -    private final Map<UUID, AtomicInteger> invalidMutations;
 +    private final Set<Keyspace> keyspacesReplayed;
 +    private final Queue<Future<Integer>> futures;
 +
      private final AtomicInteger replayedCount;
-     private final Map<UUID, ReplayPositionFilter> cfPersisted;
 -    private final Map<UUID, IntervalSet<ReplayPosition>> cfPersisted;
 -    private final ReplayPosition globalPosition;
 -    private final CRC32 checksum;
 -    private byte[] buffer;
 -    private byte[] uncompressedBuffer;
++    private final Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted;
 +    private final CommitLogPosition globalPosition;
 +
 +    // Used to throttle speed of replay of mutations if we pass the max outstanding count
 +    private long pendingMutationBytes = 0;
  
      private final ReplayFilter replayFilter;
      private final CommitLogArchiver archiver;
  
 -    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, IntervalSet<ReplayPosition>> cfPersisted, ReplayFilter replayFilter)
 +    @VisibleForTesting
 +    protected CommitLogReader commitLogReader;
 +
 +    CommitLogReplayer(CommitLog commitLog,
 +                      CommitLogPosition globalPosition,
-                       Map<UUID, ReplayPositionFilter> cfPersisted,
++                      Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted,
 +                      ReplayFilter replayFilter)
      {
 -        this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
 -        this.futures = new ArrayList<Future<?>>();
 -        this.buffer = new byte[4096];
 -        this.uncompressedBuffer = new byte[4096];
 -        this.invalidMutations = new HashMap<UUID, AtomicInteger>();
 +        this.keyspacesReplayed = new NonBlockingHashSet<Keyspace>();
 +        this.futures = new ArrayDeque<Future<Integer>>();
          // count the number of replayed mutation. We don't really care about atomicity, but we need it to be a reference.
          this.replayedCount = new AtomicInteger();
 -        this.checksum = new CRC32();
          this.cfPersisted = cfPersisted;
          this.globalPosition = globalPosition;
          this.replayFilter = replayFilter;
@@@ -89,10 -103,9 +89,10 @@@
  
      public static CommitLogReplayer construct(CommitLog commitLog)
      {
-         // compute per-CF and global commit log segment positions
-         Map<UUID, ReplayPositionFilter> cfPersisted = new HashMap<>();
+         // compute per-CF and global replay intervals
 -        Map<UUID, IntervalSet<ReplayPosition>> cfPersisted = new HashMap<>();
++        Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted = new HashMap<>();
          ReplayFilter replayFilter = ReplayFilter.create();
-         CommitLogPosition globalPosition = null;
++
          for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
          {
              // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
@@@ -117,15 -130,11 +117,11 @@@
                  }
              }
  
-             ReplayPositionFilter filter = new ReplayPositionFilter(cfs.getSSTables(), truncatedAt);
-             if (!filter.isEmpty())
-                 cfPersisted.put(cfs.metadata.cfId, filter);
-             else
-                 globalPosition = CommitLogPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
 -            IntervalSet<ReplayPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
++            IntervalSet<CommitLogPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
+             cfPersisted.put(cfs.metadata.cfId, filter);
          }
-         if (globalPosition == null)
-             globalPosition = firstNotCovered(cfPersisted.values());
-         logger.trace("Global commit log segment position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
 -        ReplayPosition globalPosition = firstNotCovered(cfPersisted.values());
++        CommitLogPosition globalPosition = firstNotCovered(cfPersisted.values());
+         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
          return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
      }
  
@@@ -174,135 -208,38 +170,105 @@@
          return replayedCount.get();
      }
  
 -    private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader, boolean tolerateTruncation) throws IOException
 +    /*
 +     * Wrapper around initiating mutations read from the log to make it possible
 +     * to spy on initiated mutations for test
 +     */
 +    @VisibleForTesting
 +    public static class MutationInitiator
      {
 -        if (offset > reader.length() - CommitLogSegment.SYNC_MARKER_SIZE)
 -        {
 -            // There was no room in the segment to write a final header. No data could be present here.
 -            return -1;
 -        }
 -        reader.seek(offset);
 -        CRC32 crc = new CRC32();
 -        updateChecksumInt(crc, (int) (descriptor.id & 0xFFFFFFFFL));
 -        updateChecksumInt(crc, (int) (descriptor.id >>> 32));
 -        updateChecksumInt(crc, (int) reader.getPosition());
 -        int end = reader.readInt();
 -        long filecrc = reader.readInt() & 0xffffffffL;
 -        if (crc.getValue() != filecrc)
 +        protected Future<Integer> initiateMutation(final Mutation mutation,
 +                                                   final long segmentId,
 +                                                   final int serializedSize,
 +                                                   final int entryLocation,
 +                                                   final CommitLogReplayer commitLogReplayer)
          {
 -            if (end != 0 || filecrc != 0)
 +            Runnable runnable = new WrappedRunnable()
              {
 -                handleReplayError(false,
 -                                  "Encountered bad header at position %d of commit log %s, with invalid CRC. " +
 -                                  "The end of segment marker should be zero.",
 -                                  offset, reader.getPath());
 -            }
 -            return -1;
 -        }
 -        else if (end < offset || end > reader.length())
 -        {
 -            handleReplayError(tolerateTruncation, "Encountered bad header at position %d of commit log %s, with bad position but valid CRC",
 -                              offset, reader.getPath());
 -            return -1;
 +                public void runMayThrow()
 +                {
 +                    if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
 +                        return;
 +                    if (commitLogReplayer.pointInTimeExceeded(mutation))
 +                        return;
 +
 +                    final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
 +
 +                    // Rebuild the mutation, omitting column families that
 +                    //    a) the user has requested that we ignore,
 +                    //    b) have already been flushed,
 +                    // or c) are part of a cf that was dropped.
 +                    // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
 +                    Mutation newMutation = null;
 +                    for (PartitionUpdate update : commitLogReplayer.replayFilter.filter(mutation))
 +                    {
 +                        if (Schema.instance.getCF(update.metadata().cfId) == null)
 +                            continue; // dropped
 +
 +                        // replay if current segment is newer than last flushed one or,
 +                        // if it is the last known segment, if we are after the commit log segment position
 +                        if (commitLogReplayer.shouldReplay(update.metadata().cfId, new CommitLogPosition(segmentId, entryLocation)))
 +                        {
 +                            if (newMutation == null)
 +                                newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
 +                            newMutation.add(update);
 +                            commitLogReplayer.replayedCount.incrementAndGet();
 +                        }
 +                    }
 +                    if (newMutation != null)
 +                    {
 +                        assert !newMutation.isEmpty();
 +
 +                        try
 +                        {
 +                            Uninterruptibles.getUninterruptibly(Keyspace.open(newMutation.getKeyspaceName()).applyFromCommitLog(newMutation));
 +                        }
 +                        catch (ExecutionException e)
 +                        {
 +                            throw Throwables.propagate(e.getCause());
 +                        }
 +
 +                        commitLogReplayer.keyspacesReplayed.add(keyspace);
 +                    }
 +                }
 +            };
 +            return StageManager.getStage(Stage.MUTATION).submit(runnable, serializedSize);
          }
 -        return end;
 +    }
 +
 +    /**
-      * A filter of known safe-to-discard commit log replay positions, based on
++     * A set of known safe-to-discard commit log replay positions, based on
 +     * the range covered by on disk sstables and those prior to the most recent truncation record
 +     */
-     public static class ReplayPositionFilter
++    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
 +    {
-         final NavigableMap<CommitLogPosition, CommitLogPosition> persisted = new TreeMap<>();
-         public ReplayPositionFilter(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
-         {
-             for (SSTableReader reader : onDisk)
-             {
-                 CommitLogPosition start = reader.getSSTableMetadata().commitLogLowerBound;
-                 CommitLogPosition end = reader.getSSTableMetadata().commitLogUpperBound;
-                 add(persisted, start, end);
-             }
-             if (truncatedAt != null)
-                 add(persisted, CommitLogPosition.NONE, truncatedAt);
-         }
++        IntervalSet.Builder<CommitLogPosition> builder = new IntervalSet.Builder<>();
++        for (SSTableReader reader : onDisk)
++            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
 +
-         private static void add(NavigableMap<CommitLogPosition, CommitLogPosition> ranges, CommitLogPosition start, CommitLogPosition end)
-         {
-             // extend ourselves to cover any ranges we overlap
-             // record directly preceding our end may extend past us, so take the max of our end and its
-             Map.Entry<CommitLogPosition, CommitLogPosition> extend = ranges.floorEntry(end);
-             if (extend != null && extend.getValue().compareTo(end) > 0)
-                 end = extend.getValue();
- 
-             // record directly preceding our start may extend into us; if it does, we take it as our start
-             extend = ranges.lowerEntry(start);
-             if (extend != null && extend.getValue().compareTo(start) >= 0)
-                 start = extend.getKey();
- 
-             ranges.subMap(start, end).clear();
-             ranges.put(start, end);
-         }
- 
-         public boolean shouldReplay(CommitLogPosition position)
-         {
-             // replay ranges are start exclusive, end inclusive
-             Map.Entry<CommitLogPosition, CommitLogPosition> range = persisted.lowerEntry(position);
-             return range == null || position.compareTo(range.getValue()) > 0;
-         }
- 
-         public boolean isEmpty()
-         {
-             return persisted.isEmpty();
-         }
++        if (truncatedAt != null)
++            builder.add(CommitLogPosition.NONE, truncatedAt);
++        return builder.build();
 +    }
 +
-     public static CommitLogPosition firstNotCovered(Iterable<ReplayPositionFilter> ranges)
++    /**
++     * Find the earliest commit log position that is not covered by the known flushed ranges for some table.
++     *
++     * For efficiency this assumes that the first contiguously flushed interval we know of contains the moment that the
++     * given table was constructed* and hence we can start replay from the end of that interval.
++     *
++     * If such an interval is not known, we must replay from the beginning.
++     *
++     * * This is not true only until if the very first flush of a table stalled or failed, while the second or latter
++     *   succeeded. The chances of this happening are at most very low, and if the assumption does prove to be
++     *   incorrect during replay there is little chance that the affected deployment is in production.
++     */
++    public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLogPosition>> ranges)
 +    {
-         CommitLogPosition min = null;
-         for (ReplayPositionFilter map : ranges)
-         {
-             CommitLogPosition first = map.persisted.firstEntry().getValue();
-             if (min == null)
-                 min = first;
-             else
-                 min = Ordering.natural().min(min, first);
-         }
-         if (min == null)
-             return CommitLogPosition.NONE;
-         return min;
++        return ranges.stream()
++                .map(intervals -> Iterables.getFirst(intervals.ends(), CommitLogPosition.NONE)) 
++                .min(Ordering.natural())
++                .get(); // iteration is per known-CF, there must be at least one. 
      }
  
      abstract static class ReplayFilter
@@@ -386,12 -323,346 +352,11 @@@
       *
       * @return true iff replay is necessary
       */
 -    private boolean shouldReplay(UUID cfId, ReplayPosition position)
 +    private boolean shouldReplay(UUID cfId, CommitLogPosition position)
      {
-         ReplayPositionFilter filter = cfPersisted.get(cfId);
-         return filter == null || filter.shouldReplay(position);
+         return !cfPersisted.get(cfId).contains(position);
      }
  
 -    @SuppressWarnings("resource")
 -    public void recover(File file, boolean tolerateTruncation) throws IOException
 -    {
 -        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
 -        try(ChannelProxy channel = new ChannelProxy(file);
 -            RandomAccessReader reader = RandomAccessReader.open(channel))
 -        {
 -            if (desc.version < CommitLogDescriptor.VERSION_21)
 -            {
 -                if (logAndCheckIfShouldSkip(file, desc))
 -                    return;
 -                if (globalPosition.segment == desc.id)
 -                    reader.seek(globalPosition.position);
 -                replaySyncSection(reader, (int) reader.length(), desc, desc.fileName(), tolerateTruncation);
 -                return;
 -            }
 -
 -            final long segmentId = desc.id;
 -            try
 -            {
 -                desc = CommitLogDescriptor.readHeader(reader);
 -            }
 -            catch (IOException e)
 -            {
 -                desc = null;
 -            }
 -            if (desc == null) {
 -                handleReplayError(false, "Could not read commit log descriptor in file %s", file);
 -                return;
 -            }
 -            if (segmentId != desc.id)
 -            {
 -                handleReplayError(false, "Segment id mismatch (filename %d, descriptor %d) in file %s", segmentId, desc.id, file);
 -                // continue processing if ignored.
 -            }
 -
 -            if (logAndCheckIfShouldSkip(file, desc))
 -                return;
 -
 -            ICompressor compressor = null;
 -            if (desc.compression != null)
 -            {
 -                try
 -                {
 -                    compressor = CompressionParams.createCompressor(desc.compression);
 -                }
 -                catch (ConfigurationException e)
 -                {
 -                    handleReplayError(false, "Unknown compression: %s", e.getMessage());
 -                    return;
 -                }
 -            }
 -
 -            assert reader.length() <= Integer.MAX_VALUE;
 -            int end = (int) reader.getFilePointer();
 -            int replayEnd = end;
 -
 -            while ((end = readSyncMarker(desc, end, reader, tolerateTruncation)) >= 0)
 -            {
 -                int replayPos = replayEnd + CommitLogSegment.SYNC_MARKER_SIZE;
 -
 -                if (logger.isTraceEnabled())
 -                    logger.trace("Replaying {} between {} and {}", file, reader.getFilePointer(), end);
 -                if (compressor != null)
 -                {
 -                    int uncompressedLength = reader.readInt();
 -                    replayEnd = replayPos + uncompressedLength;
 -                }
 -                else
 -                {
 -                    replayEnd = end;
 -                }
 -
 -                if (segmentId == globalPosition.segment && replayEnd < globalPosition.position)
 -                    // Skip over flushed section.
 -                    continue;
 -
 -                FileDataInput sectionReader = reader;
 -                String errorContext = desc.fileName();
 -                // In the uncompressed case the last non-fully-flushed section can be anywhere in the file.
 -                boolean tolerateErrorsInSection = tolerateTruncation;
 -                if (compressor != null)
 -                {
 -                    // In the compressed case we know if this is the last section.
 -                    tolerateErrorsInSection &= end == reader.length() || end < 0;
 -
 -                    int start = (int) reader.getFilePointer();
 -                    try
 -                    {
 -                        int compressedLength = end - start;
 -                        if (logger.isTraceEnabled())
 -                            logger.trace("Decompressing {} between replay positions {} and {}",
 -                                         file,
 -                                         replayPos,
 -                                         replayEnd);
 -                        if (compressedLength > buffer.length)
 -                            buffer = new byte[(int) (1.2 * compressedLength)];
 -                        reader.readFully(buffer, 0, compressedLength);
 -                        int uncompressedLength = replayEnd - replayPos;
 -                        if (uncompressedLength > uncompressedBuffer.length)
 -                            uncompressedBuffer = new byte[(int) (1.2 * uncompressedLength)];
 -                        compressedLength = compressor.uncompress(buffer, 0, compressedLength, uncompressedBuffer, 0);
 -                        sectionReader = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer), reader.getPath(), replayPos);
 -                        errorContext = "compressed section at " + start + " in " + errorContext;
 -                    }
 -                    catch (IOException | ArrayIndexOutOfBoundsException e)
 -                    {
 -                        handleReplayError(tolerateErrorsInSection,
 -                                          "Unexpected exception decompressing section at %d: %s",
 -                                          start, e);
 -                        continue;
 -                    }
 -                }
 -
 -                if (!replaySyncSection(sectionReader, replayEnd, desc, errorContext, tolerateErrorsInSection))
 -                    break;
 -            }
 -            logger.debug("Finished reading {}", file);
 -        }
 -    }
 -
 -    public boolean logAndCheckIfShouldSkip(File file, CommitLogDescriptor desc)
 -    {
 -        logger.debug("Replaying {} (CL version {}, messaging version {}, compression {})",
 -                    file.getPath(),
 -                    desc.version,
 -                    desc.getMessagingVersion(),
 -                    desc.compression);
 -
 -        if (globalPosition.segment > desc.id)
 -        {
 -            logger.trace("skipping replay of fully-flushed {}", file);
 -            return true;
 -        }
 -        return false;
 -    }
 -
 -    /**
 -     * Replays a sync section containing a list of mutations.
 -     *
 -     * @return Whether replay should continue with the next section.
 -     */
 -    private boolean replaySyncSection(FileDataInput reader, int end, CommitLogDescriptor desc, String errorContext, boolean tolerateErrors) throws IOException
 -    {
 -         /* read the logs populate Mutation and apply */
 -        while (reader.getFilePointer() < end && !reader.isEOF())
 -        {
 -            long mutationStart = reader.getFilePointer();
 -            if (logger.isTraceEnabled())
 -                logger.trace("Reading mutation at {}", mutationStart);
 -
 -            long claimedCRC32;
 -            int serializedSize;
 -            try
 -            {
 -                // any of the reads may hit EOF
 -                serializedSize = reader.readInt();
 -                if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
 -                {
 -                    logger.trace("Encountered end of segment marker at {}", reader.getFilePointer());
 -                    return false;
 -                }
 -
 -                // Mutation must be at LEAST 10 bytes:
 -                // 3 each for a non-empty Keyspace and Key (including the
 -                // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
 -                // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
 -                if (serializedSize < 10)
 -                {
 -                    handleReplayError(tolerateErrors,
 -                                      "Invalid mutation size %d at %d in %s",
 -                                      serializedSize, mutationStart, errorContext);
 -                    return false;
 -                }
 -
 -                long claimedSizeChecksum;
 -                if (desc.version < CommitLogDescriptor.VERSION_21)
 -                    claimedSizeChecksum = reader.readLong();
 -                else
 -                    claimedSizeChecksum = reader.readInt() & 0xffffffffL;
 -                checksum.reset();
 -                if (desc.version < CommitLogDescriptor.VERSION_20)
 -                    checksum.update(serializedSize);
 -                else
 -                    updateChecksumInt(checksum, serializedSize);
 -
 -                if (checksum.getValue() != claimedSizeChecksum)
 -                {
 -                    handleReplayError(tolerateErrors,
 -                                      "Mutation size checksum failure at %d in %s",
 -                                      mutationStart, errorContext);
 -                    return false;
 -                }
 -                // ok.
 -
 -                if (serializedSize > buffer.length)
 -                    buffer = new byte[(int) (1.2 * serializedSize)];
 -                reader.readFully(buffer, 0, serializedSize);
 -                if (desc.version < CommitLogDescriptor.VERSION_21)
 -                    claimedCRC32 = reader.readLong();
 -                else
 -                    claimedCRC32 = reader.readInt() & 0xffffffffL;
 -            }
 -            catch (EOFException eof)
 -            {
 -                handleReplayError(tolerateErrors,
 -                                  "Unexpected end of segment",
 -                                  mutationStart, errorContext);
 -                return false; // last CL entry didn't get completely written. that's ok.
 -            }
 -
 -            checksum.update(buffer, 0, serializedSize);
 -            if (claimedCRC32 != checksum.getValue())
 -            {
 -                handleReplayError(tolerateErrors,
 -                                  "Mutation checksum failure at %d in %s",
 -                                  mutationStart, errorContext);
 -                continue;
 -            }
 -            replayMutation(buffer, serializedSize, (int) reader.getFilePointer(), desc);
 -        }
 -        return true;
 -    }
 -
 -    /**
 -     * Deserializes and replays a commit log entry.
 -     */
 -    void replayMutation(byte[] inputBuffer, int size,
 -            final int entryLocation, final CommitLogDescriptor desc) throws IOException
 -    {
 -
 -        final Mutation mutation;
 -        try (RebufferingInputStream bufIn = new DataInputBuffer(inputBuffer, 0, size))
 -        {
 -            mutation = Mutation.serializer.deserialize(bufIn,
 -                                                       desc.getMessagingVersion(),
 -                                                       SerializationHelper.Flag.LOCAL);
 -            // doublecheck that what we read is [still] valid for the current schema
 -            for (PartitionUpdate upd : mutation.getPartitionUpdates())
 -                upd.validate();
 -        }
 -        catch (UnknownColumnFamilyException ex)
 -        {
 -            if (ex.cfId == null)
 -                return;
 -            AtomicInteger i = invalidMutations.get(ex.cfId);
 -            if (i == null)
 -            {
 -                i = new AtomicInteger(1);
 -                invalidMutations.put(ex.cfId, i);
 -            }
 -            else
 -                i.incrementAndGet();
 -            return;
 -        }
 -        catch (Throwable t)
 -        {
 -            JVMStabilityInspector.inspectThrowable(t);
 -            File f = File.createTempFile("mutation", "dat");
 -
 -            try (DataOutputStream out = new DataOutputStream(new FileOutputStream(f)))
 -            {
 -                out.write(inputBuffer, 0, size);
 -            }
 -
 -            // Checksum passed so this error can't be permissible.
 -            handleReplayError(false,
 -                              "Unexpected error deserializing mutation; saved to %s.  " +
 -                              "This may be caused by replaying a mutation against a table with the same name but incompatible schema.  " +
 -                              "Exception follows: %s",
 -                              f.getAbsolutePath(),
 -                              t);
 -            return;
 -        }
 -
 -        if (logger.isTraceEnabled())
 -            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), mutation.key(), "{" + StringUtils.join(mutation.getPartitionUpdates().iterator(), ", ") + "}");
 -
 -        Runnable runnable = new WrappedRunnable()
 -        {
 -            public void runMayThrow()
 -            {
 -                if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
 -                    return;
 -                if (pointInTimeExceeded(mutation))
 -                    return;
 -
 -                final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
 -
 -                // Rebuild the mutation, omitting column families that
 -                //    a) the user has requested that we ignore,
 -                //    b) have already been flushed,
 -                // or c) are part of a cf that was dropped.
 -                // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
 -                Mutation newMutation = null;
 -                for (PartitionUpdate update : replayFilter.filter(mutation))
 -                {
 -                    if (Schema.instance.getCF(update.metadata().cfId) == null)
 -                        continue; // dropped
 -
 -                    // replay if current segment is newer than last flushed one or,
 -                    // if it is the last known segment, if we are after the replay position
 -                    if (shouldReplay(update.metadata().cfId, new ReplayPosition(desc.id, entryLocation)))
 -                    {
 -                        if (newMutation == null)
 -                            newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
 -                        newMutation.add(update);
 -                        replayedCount.incrementAndGet();
 -                    }
 -                }
 -                if (newMutation != null)
 -                {
 -                    assert !newMutation.isEmpty();
 -
 -                    try
 -                    {
 -                        Uninterruptibles.getUninterruptibly(Keyspace.open(newMutation.getKeyspaceName()).applyFromCommitLog(newMutation));
 -                    }
 -                    catch (ExecutionException e)
 -                    {
 -                        throw Throwables.propagate(e.getCause());
 -                    }
 -
 -                    keyspacesRecovered.add(keyspace);
 -                }
 -            }
 -        };
 -        futures.add(StageManager.getStage(Stage.MUTATION).submit(runnable));
 -        if (futures.size() > MAX_OUTSTANDING_REPLAY_COUNT)
 -        {
 -            FBUtilities.waitOnFutures(futures);
 -            futures.clear();
 -        }
 -    }
 -
      protected boolean pointInTimeExceeded(Mutation fm)
      {
          long restoreTarget = archiver.restorePointInTime;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index a1158be,d2f12bf..e32c204
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@@ -32,13 -40,16 +32,14 @@@ import org.cliffc.high_scale_lib.NonBlo
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.config.CFMetaData;
 -import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.Schema;
 +import com.codahale.metrics.Timer;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.db.Mutation;
 +import org.apache.cassandra.db.commitlog.CommitLog.Configuration;
  import org.apache.cassandra.db.partitions.PartitionUpdate;
  import org.apache.cassandra.io.FSWriteError;
 -import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.utils.CLibrary;
+ import org.apache.cassandra.utils.IntegerInterval;
  import org.apache.cassandra.utils.concurrent.OpOrder;
  import org.apache.cassandra.utils.concurrent.WaitQueue;
  
@@@ -461,22 -448,18 +475,19 @@@ public abstract class CommitLogSegmen
       * given context argument is contained in this file, it will only mark the CF as
       * clean if no newer writes have taken place.
       *
--     * @param cfId    the column family ID that is now clean
--     * @param context the optional clean offset
++     * @param cfId           the column family ID that is now clean
++     * @param startPosition  the start of the range that is clean
++     * @param endPosition    the end of the range that is clean
       */
-     public synchronized void markClean(UUID cfId, CommitLogPosition context)
 -    public synchronized void markClean(UUID cfId, ReplayPosition startPosition, ReplayPosition endPosition)
++    public synchronized void markClean(UUID cfId, CommitLogPosition startPosition, CommitLogPosition endPosition)
      {
 -        if (startPosition.segment > id || endPosition.segment < id)
++        if (startPosition.segmentId > id || endPosition.segmentId < id)
+             return;
          if (!cfDirty.containsKey(cfId))
              return;
-         if (context.segmentId == id)
-             markClean(cfId, context.position);
-         else if (context.segmentId > id)
-             markClean(cfId, Integer.MAX_VALUE);
-     }
- 
-     private void markClean(UUID cfId, int position)
-     {
-         ensureAtleast(cfClean, cfId, position);
 -        int start = startPosition.segment == id ? startPosition.position : 0;
 -        int end = endPosition.segment == id ? endPosition.position : Integer.MAX_VALUE;
++        int start = startPosition.segmentId == id ? startPosition.position : 0;
++        int end = endPosition.segmentId == id ? endPosition.position : Integer.MAX_VALUE;
+         cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
          removeCleanFromDirty();
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------


[07/23] cassandra git commit: Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

* commit 'bd6654733dded3513c2c7acf96df2c364b0c043e':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-3.9
Commit: 6dc1745edd8d3861d853ee56f49ac67633a753b0
Parents: 0398521 bd66547
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:36:29 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:11 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  67 +++++---
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  95 +++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 336 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 87228d3,1275631..7fcf373
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,13 +1,58 @@@
 +2.2.8
 + * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
 + * Revert CASSANDRA-11427 (CASSANDRA-12351)
 + * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
 + * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
 + * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 + * Synchronize ThriftServer::stop() (CASSANDRA-12105)
 + * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 + * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
 + * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 + * Don't write shadowed range tombstone (CASSANDRA-12030)
 +Merged from 2.1:
++=======
+ 2.1.16
+  * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
   * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
   * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 + * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.1.15
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d0cb200,6e82745..0835a28
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -891,12 -873,20 +895,30 @@@ public class ColumnFamilyStore implemen
      {
          synchronized (data)
          {
+             if (previousFlushFailure != null)
+                 throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                               + "no flushes can be carried out until the node is restarted.",
+                                                 previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
-             flushExecutor.execute(flush);
+             ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+             flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
              postFlushExecutor.submit(task);
-             return task;
+ 
+             @SuppressWarnings("unchecked")
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future = 
++                    // If either of the two tasks errors out, resulting future must also error out.
++                    // Combine the two futures and only return post-flush result after both have completed.
++                    Futures.transform(Futures.allAsList(flushTask, task),
++                                      new Function<List<Object>, ReplayPosition>()
++                                      {
++                                          public ReplayPosition apply(List<Object> input)
++                                          {
++                                              return (ReplayPosition) input.get(1);
++                                          }
++                                      });
+             return future;
          }
      }
  
@@@ -999,13 -978,9 +1021,12 @@@
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
 -        final ReplayPosition lastReplayPosition;
 +        final ReplayPosition commitLogUpperBound;
 +        final List<Memtable> memtables;
 +        final List<SSTableReader> readers;
-         volatile FSWriteError flushFailure = null;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
 +        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
 +                          List<Memtable> memtables, List<SSTableReader> readers)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
@@@ -1049,23 -1019,14 +1070,17 @@@
                  throw new IllegalStateException();
              }
  
-             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
 -            // must check lastReplayPosition != null because Flush may find that all memtables are clean
 -            // and so not set a lastReplayPosition
 -            if (lastReplayPosition != null)
++            CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
++            for (int i = 0 ; i < memtables.size() ; i++)
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     SSTableReader reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
-                 }
 -                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
++                Memtable memtable = memtables.get(i);
++                SSTableReader reader = readers.get(i);
++                memtable.cfs.data.permitCompactionOfFlushed(reader);
++                memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
              }
 -
              metric.pendingFlushes.dec();
 +
-             if (flushFailure != null)
-                 throw flushFailure;
 +            return commitLogUpperBound;
          }
      }
  
@@@ -1162,11 -1131,13 +1177,15 @@@
                  for (Memtable memtable : memtables)
                  {
                      // flush the memtable
 -                    MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
 +                    SSTableReader reader = memtable.flush();
 +                    memtable.cfs.data.replaceFlushed(memtable, reader);
                      reclaim(memtable);
 +                    readers.add(reader);
                  }
+ 
+                 // signal the post-flush we've done our work
+                 // Note: This should not be done in case of error. Read more below.
+                 postFlush.latch.countDown();
              }
              catch (FSWriteError e)
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
index 0000000,c0023dc..826d6e6
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@@ -1,0 -1,87 +1,95 @@@
+ /*
+  * 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.cql3;
+ 
+ import static junit.framework.Assert.fail;
+ 
+ import java.io.IOError;
+ import java.util.UUID;
+ import java.util.concurrent.ExecutionException;
+ 
++import org.junit.After;
+ import org.junit.Assert;
+ 
+ import org.apache.cassandra.db.BlacklistedDirectories;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Directories.DataDirectory;
+ import org.apache.cassandra.db.commitlog.CommitLog;
+ import org.apache.cassandra.db.commitlog.CommitLogSegment;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.io.FSWriteError;
+ 
+ /**
+  * Test that exceptions during flush are treated according to the disk failure policy.
+  * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+  */
+ public class OutOfSpaceBase extends CQLTester
+ {
+     public void makeTable() throws Throwable
+     {
+         createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+     }
+ 
+     public void markDirectoriesUnwriteable()
+     {
+         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+     }
+ 
+     public void flushAndExpectError() throws InterruptedException, ExecutionException
+     {
+         try
+         {
+             Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+             fail("FSWriteError expected.");
+         }
+         catch (ExecutionException e)
+         {
+             // Correct path.
+             Assert.assertTrue(e.getCause() instanceof FSWriteError);
+         }
+ 
+         // Make sure commit log wasn't discarded.
+         UUID cfid = currentTableMetadata().cfId;
+         for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+             if (segment.getDirtyCFIDs().contains(cfid))
+                 return;
+         fail("Expected commit log to remain dirty for the affected table.");
+     }
++
++
++    @After
++    public void afterTest() throws Throwable
++    {
++        // Override CQLTester's afterTest method; clean-up will fail due to flush failing.
++    }
+ }


[16/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
Change commitlog and sstables to track dirty and clean intervals.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828


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

Branch: refs/heads/cassandra-3.0
Commit: 904cb5d10e0de1a6ca89249be8c257ed38a80ef0
Parents: cf85f52
Author: Branimir Lambov <br...@datastax.com>
Authored: Sat May 14 11:31:16 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:38:37 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  66 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  18 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         |  59 +++-
 .../db/commitlog/CommitLogSegment.java          |  77 ++---
 .../db/commitlog/CommitLogSegmentManager.java   |   4 +-
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../cassandra/db/commitlog/ReplayPosition.java  |  71 ----
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  44 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  36 +-
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  14 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  38 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   2 +-
 test/unit/org/apache/cassandra/Util.java        |  21 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 159 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 98 files changed, 1229 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 046c8b3..b596fc9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.9
+ * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
  * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
  * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
  * Fix upgrade of super columns on thrift (CASSANDRA-12335)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
index f47fd57..bc733d7 100644
--- a/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
+++ b/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
@@ -29,6 +29,8 @@ import java.util.concurrent.CopyOnWriteArraySet;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
+import com.google.common.annotations.VisibleForTesting;
+
 import org.apache.cassandra.utils.JVMStabilityInspector;
 
 public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
@@ -101,6 +103,17 @@ public class BlacklistedDirectories implements BlacklistedDirectoriesMBean
     }
 
     /**
+     * Testing only!
+     * Clear the set of unwritable directories.
+     */
+    @VisibleForTesting
+    public static void clearUnwritableUnsafe()
+    {
+        instance.unwritableDirectories.clear();
+    }
+
+
+    /**
      * Tells whether or not the directory is blacklisted for reads.
      * @return whether or not the directory is blacklisted for reads.
      */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 400fd36..82604e2 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -179,9 +179,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    public static volatile ColumnFamilyStore discardFlushResults;
-
     public final Keyspace keyspace;
     public final String name;
     public final CFMetaData metadata;
@@ -926,25 +923,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         volatile FSWriteError flushFailure = null;
-        final ReplayPosition commitLogUpperBound;
         final List<Memtable> memtables;
-        final List<Collection<SSTableReader>> readers;
 
-        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
-                          List<Memtable> memtables, List<Collection<SSTableReader>> readers)
+        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier,
+                          List<Memtable> memtables)
         {
             this.writeBarrier = writeBarrier;
             this.flushSecondaryIndexes = flushSecondaryIndexes;
-            this.commitLogUpperBound = commitLogUpperBound;
             this.memtables = memtables;
-            this.readers = readers;
         }
 
         public ReplayPosition call()
         {
-            if (discardFlushResults == ColumnFamilyStore.this)
-                return commitLogUpperBound;
-
             writeBarrier.await();
 
             /**
@@ -968,17 +958,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 throw new IllegalStateException();
             }
 
+            ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (flushFailure == null)
+            if (flushFailure == null && !memtables.isEmpty())
             {
-                CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                for (int i = 0 ; i < memtables.size() ; i++)
-                {
-                    Memtable memtable = memtables.get(i);
-                    Collection<SSTableReader> reader = readers.get(i);
-                    memtable.cfs.data.permitCompactionOfFlushed(reader);
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, reader);
-                }
+                Memtable memtable = memtables.get(0);
+                commitLogUpperBound = memtable.getCommitLogUpperBound();
+                CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
             }
 
             metric.pendingFlushes.dec();
@@ -1002,7 +988,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         final OpOrder.Barrier writeBarrier;
         final List<Memtable> memtables = new ArrayList<>();
-        final List<Collection<SSTableReader>> readers = new ArrayList<>();
         final PostFlush postFlush;
         final boolean truncate;
 
@@ -1044,7 +1029,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
             // replay positions have also completed, i.e. the memtables are done and ready to flush
             writeBarrier.issue();
-            postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
+            postFlush = new PostFlush(!truncate, writeBarrier, memtables);
         }
 
         public void run()
@@ -1063,8 +1048,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 memtable.cfs.data.markFlushing(memtable);
                 if (memtable.isClean() || truncate)
                 {
-                    memtable.cfs.data.replaceFlushed(memtable, Collections.emptyList());
-                    memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, Collections.emptyList());
+                    memtable.cfs.replaceFlushed(memtable, Collections.emptyList());
                     reclaim(memtable);
                     iter.remove();
                 }
@@ -1077,9 +1061,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                 for (Memtable memtable : memtables)
                 {
                     Collection<SSTableReader> readers = memtable.flush();
-                    memtable.cfs.data.replaceFlushed(memtable, readers);
+                    memtable.cfs.replaceFlushed(memtable, readers);
                     reclaim(memtable);
-                    this.readers.add(readers);
                 }
             }
             catch (FSWriteError e)
@@ -1126,21 +1109,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         }
     }
 
-    @VisibleForTesting
-    // this method should ONLY be used for testing commit log behaviour; it discards the current memtable
-    // contents without marking the commit log clean, and prevents any proceeding flushes from marking
-    // the commit log as done, however they *will* terminate (unlike under typical failures) to ensure progress is made
-    public void simulateFailedFlush()
-    {
-        discardFlushResults = this;
-        data.markFlushing(data.switchMemtable(false, new Memtable(new AtomicReference<>(CommitLog.instance.getContext()), this)));
-    }
-
-    public void resumeFlushing()
-    {
-        discardFlushResults = null;
-    }
-
     /**
      * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
      * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
@@ -1483,16 +1451,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return data;
     }
 
-    public Collection<SSTableReader> getSSTables()
-    {
-        return data.getSSTables();
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompactSSTables()
-    {
-        return data.getPermittedToCompact();
-    }
-
     public Set<SSTableReader> getLiveSSTables()
     {
         return data.getView().liveSSTables();
@@ -2032,7 +1990,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         long now = System.currentTimeMillis();
         // make sure none of our sstables are somehow in the future (clock drift, perhaps)
         for (ColumnFamilyStore cfs : concatWithIndexes())
-            for (SSTableReader sstable : cfs.data.getSSTables())
+            for (SSTableReader sstable : cfs.getLiveSSTables())
                 now = Math.max(now, sstable.maxDataAge);
         truncatedAt = now;
 
@@ -2130,7 +2088,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             public LifecycleTransaction call() throws Exception
             {
                 assert data.getCompacting().isEmpty() : data.getCompacting();
-                Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                Iterable<SSTableReader> sstables = getLiveSSTables();
                 sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
                 sstables = ImmutableList.copyOf(sstables);
                 LifecycleTransaction modifier = data.tryModify(sstables, operationType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 01ffd52..877f984 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -385,7 +385,7 @@ public class Directories
 
         if (candidates.isEmpty())
             if (tooBig)
-                throw new RuntimeException("Insufficient disk space to write " + writeSize + " bytes");
+                throw new FSWriteError(new IOException("Insufficient disk space to write " + writeSize + " bytes"), "");
             else
                 throw new FSWriteError(new IOException("All configured data directories have been blacklisted as unwritable for erroring out"), "");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 93dc5af..3c77092 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
@@ -193,6 +194,11 @@ public class Memtable implements Comparable<Memtable>
         return commitLogLowerBound.get();
     }
 
+    public ReplayPosition getCommitLogUpperBound()
+    {
+        return commitLogUpperBound.get();
+    }
+
     public boolean isLive()
     {
         return allocator.isLive();
@@ -331,6 +337,15 @@ public class Memtable implements Comparable<Memtable>
         return minTimestamp;
     }
 
+    /**
+     * For testing only. Give this memtable too big a size to make it always fail flushing.
+     */
+    @VisibleForTesting
+    public void makeUnflushable()
+    {
+        liveDataSize.addAndGet(1L * 1024 * 1024 * 1024 * 1024 * 1024);
+    }
+
     private long estimatedSize()
     {
         long keySize = 0;
@@ -418,8 +433,7 @@ public class Memtable implements Comparable<Memtable>
         {
             txn = LifecycleTransaction.offline(OperationType.FLUSH);
             MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
-                                                         .commitLogLowerBound(commitLogLowerBound.get())
-                                                         .commitLogUpperBound(commitLogUpperBound.get());
+                    .commitLogIntervals(new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));
 
             return new SSTableTxnWriter(txn,
                                         cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index dcdd855..dfe3f91 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -290,11 +290,12 @@ public class CommitLog implements CommitLogMBean
      * given. Discards any commit log segments that are no longer used.
      *
      * @param cfId    the column family ID that was flushed
-     * @param context the replay position of the flush
+     * @param lowerBound the lowest covered replay position of the flush
+     * @param lowerBound the highest covered replay position of the flush
      */
-    public void discardCompletedSegments(final UUID cfId, final ReplayPosition context)
+    public void discardCompletedSegments(final UUID cfId, final ReplayPosition lowerBound, final ReplayPosition upperBound)
     {
-        logger.trace("discard completed log segments for {}, table {}", context, cfId);
+        logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
 
         // Go thru the active segment files, which are ordered oldest to newest, marking the
         // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
@@ -303,7 +304,7 @@ public class CommitLog implements CommitLogMBean
         for (Iterator<CommitLogSegment> iter = allocator.getActiveSegments().iterator(); iter.hasNext();)
         {
             CommitLogSegment segment = iter.next();
-            segment.markClean(cfId, context);
+            segment.markClean(cfId, lowerBound, upperBound);
 
             if (segment.isUnused())
             {
@@ -318,7 +319,7 @@ public class CommitLog implements CommitLogMBean
 
             // Don't mark or try to delete any newer segments once we've reached the one containing the
             // position of the flush.
-            if (segment.contains(context))
+            if (segment.contains(upperBound))
                 break;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index f45a47a..af8efb4 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -35,6 +35,7 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Multimap;
+import com.google.common.collect.Ordering;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.commons.lang3.StringUtils;
@@ -52,6 +53,7 @@ import org.apache.cassandra.io.util.FileSegmentInputStream;
 import org.apache.cassandra.io.util.RebufferingInputStream;
 import org.apache.cassandra.schema.CompressionParams;
 import org.apache.cassandra.io.compress.ICompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.ChannelProxy;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.FileDataInput;
@@ -74,7 +76,7 @@ public class CommitLogReplayer
     private final List<Future<?>> futures;
     private final Map<UUID, AtomicInteger> invalidMutations;
     private final AtomicInteger replayedCount;
-    private final Map<UUID, ReplayPosition.ReplayFilter> cfPersisted;
+    private final Map<UUID, IntervalSet<ReplayPosition>> cfPersisted;
     private final ReplayPosition globalPosition;
     private final CRC32 checksum;
     private byte[] buffer;
@@ -83,7 +85,7 @@ public class CommitLogReplayer
     private final ReplayFilter replayFilter;
     private final CommitLogArchiver archiver;
 
-    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, ReplayPosition.ReplayFilter> cfPersisted, ReplayFilter replayFilter)
+    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, IntervalSet<ReplayPosition>> cfPersisted, ReplayFilter replayFilter)
     {
         this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
         this.futures = new ArrayList<Future<?>>();
@@ -101,10 +103,9 @@ public class CommitLogReplayer
 
     public static CommitLogReplayer construct(CommitLog commitLog)
     {
-        // compute per-CF and global replay positions
-        Map<UUID, ReplayPosition.ReplayFilter> cfPersisted = new HashMap<>();
+        // compute per-CF and global replay intervals
+        Map<UUID, IntervalSet<ReplayPosition>> cfPersisted = new HashMap<>();
         ReplayFilter replayFilter = ReplayFilter.create();
-        ReplayPosition globalPosition = null;
         for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
         {
             // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
@@ -129,14 +130,10 @@ public class CommitLogReplayer
                 }
             }
 
-            ReplayPosition.ReplayFilter filter = new ReplayPosition.ReplayFilter(cfs.getSSTables(), truncatedAt);
-            if (!filter.isEmpty())
-                cfPersisted.put(cfs.metadata.cfId, filter);
-            else
-                globalPosition = ReplayPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
+            IntervalSet<ReplayPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
+            cfPersisted.put(cfs.metadata.cfId, filter);
         }
-        if (globalPosition == null)
-            globalPosition = ReplayPosition.firstNotCovered(cfPersisted.values());
+        ReplayPosition globalPosition = firstNotCovered(cfPersisted.values());
         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
         return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
     }
@@ -148,6 +145,41 @@ public class CommitLogReplayer
             recover(clogs[i], i + 1 == clogs.length);
     }
 
+    /**
+     * A set of known safe-to-discard commit log replay positions, based on
+     * the range covered by on disk sstables and those prior to the most recent truncation record
+     */
+    public static IntervalSet<ReplayPosition> persistedIntervals(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
+    {
+        IntervalSet.Builder<ReplayPosition> builder = new IntervalSet.Builder<>();
+        for (SSTableReader reader : onDisk)
+            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
+
+        if (truncatedAt != null)
+            builder.add(ReplayPosition.NONE, truncatedAt);
+        return builder.build();
+    }
+
+    /**
+     * Find the earliest commit log position that is not covered by the known flushed ranges for some table.
+     *
+     * For efficiency this assumes that the first contiguously flushed interval we know of contains the moment that the
+     * given table was constructed* and hence we can start replay from the end of that interval.
+     *
+     * If such an interval is not known, we must replay from the beginning.
+     *
+     * * This is not true only until if the very first flush of a table stalled or failed, while the second or latter
+     *   succeeded. The chances of this happening are at most very low, and if the assumption does prove to be
+     *   incorrect during replay there is little chance that the affected deployment is in production.
+     */
+    public static ReplayPosition firstNotCovered(Collection<IntervalSet<ReplayPosition>> ranges)
+    {
+        return ranges.stream()
+                .map(intervals -> Iterables.getFirst(intervals.ends(), ReplayPosition.NONE)) 
+                .min(Ordering.natural())
+                .get(); // iteration is per known-CF, there must be at least one. 
+    }
+
     public int blockForWrites()
     {
         for (Map.Entry<UUID, AtomicInteger> entry : invalidMutations.entrySet())
@@ -293,8 +325,7 @@ public class CommitLogReplayer
      */
     private boolean shouldReplay(UUID cfId, ReplayPosition position)
     {
-        ReplayPosition.ReplayFilter filter = cfPersisted.get(cfId);
-        return filter == null || filter.shouldReplay(position);
+        return !cfPersisted.get(cfId).contains(position);
     }
 
     @SuppressWarnings("resource")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 27c05b4..d2f12bf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -49,6 +49,7 @@ import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.CLibrary;
+import org.apache.cassandra.utils.IntegerInterval;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.concurrent.WaitQueue;
 
@@ -101,11 +102,11 @@ public abstract class CommitLogSegment
     // a signal for writers to wait on to confirm the log message they provided has been written to disk
     private final WaitQueue syncComplete = new WaitQueue();
 
-    // a map of Cf->dirty position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final NonBlockingHashMap<UUID, AtomicInteger> cfDirty = new NonBlockingHashMap<>(1024);
+    // a map of Cf->dirty interval in this segment; if interval is not covered by the clean set, the log contains unflushed data
+    private final NonBlockingHashMap<UUID, IntegerInterval> cfDirty = new NonBlockingHashMap<>(1024);
 
-    // a map of Cf->clean position; this is used to permit marking Cfs clean whilst the log is still in use
-    private final ConcurrentHashMap<UUID, AtomicInteger> cfClean = new ConcurrentHashMap<>();
+    // a map of Cf->clean intervals; separate map from above to permit marking Cfs clean whilst the log is still in use
+    private final ConcurrentHashMap<UUID, IntegerInterval.Set> cfClean = new ConcurrentHashMap<>();
 
     public final long id;
 
@@ -423,10 +424,23 @@ public abstract class CommitLogSegment
         }
     }
 
+    public static<K> void coverInMap(ConcurrentMap<K, IntegerInterval> map, K key, int value)
+    {
+        IntegerInterval i = map.get(key);
+        if (i == null)
+        {
+            i = map.putIfAbsent(key, new IntegerInterval(value, value));
+            if (i == null)
+                // success
+                return;
+        }
+        i.expandToCover(value);
+    }
+
     void markDirty(Mutation mutation, int allocatedPosition)
     {
         for (PartitionUpdate update : mutation.getPartitionUpdates())
-            ensureAtleast(cfDirty, update.metadata().cfId, allocatedPosition);
+            coverInMap(cfDirty, update.metadata().cfId, allocatedPosition);
     }
 
     /**
@@ -437,55 +451,32 @@ public abstract class CommitLogSegment
      * @param cfId    the column family ID that is now clean
      * @param context the optional clean offset
      */
-    public synchronized void markClean(UUID cfId, ReplayPosition context)
+    public synchronized void markClean(UUID cfId, ReplayPosition startPosition, ReplayPosition endPosition)
     {
+        if (startPosition.segment > id || endPosition.segment < id)
+            return;
         if (!cfDirty.containsKey(cfId))
             return;
-        if (context.segment == id)
-            markClean(cfId, context.position);
-        else if (context.segment > id)
-            markClean(cfId, Integer.MAX_VALUE);
-    }
-
-    private void markClean(UUID cfId, int position)
-    {
-        ensureAtleast(cfClean, cfId, position);
+        int start = startPosition.segment == id ? startPosition.position : 0;
+        int end = endPosition.segment == id ? endPosition.position : Integer.MAX_VALUE;
+        cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
         removeCleanFromDirty();
     }
 
-    private static void ensureAtleast(ConcurrentMap<UUID, AtomicInteger> map, UUID cfId, int value)
-    {
-        AtomicInteger i = map.get(cfId);
-        if (i == null)
-        {
-            AtomicInteger i2 = map.putIfAbsent(cfId, i = new AtomicInteger());
-            if (i2 != null)
-                i = i2;
-        }
-        while (true)
-        {
-            int cur = i.get();
-            if (cur > value)
-                break;
-            if (i.compareAndSet(cur, value))
-                break;
-        }
-    }
-
     private void removeCleanFromDirty()
     {
         // if we're still allocating from this segment, don't touch anything since it can't be done thread-safely
         if (isStillAllocating())
             return;
 
-        Iterator<Map.Entry<UUID, AtomicInteger>> iter = cfClean.entrySet().iterator();
+        Iterator<Map.Entry<UUID, IntegerInterval.Set>> iter = cfClean.entrySet().iterator();
         while (iter.hasNext())
         {
-            Map.Entry<UUID, AtomicInteger> clean = iter.next();
+            Map.Entry<UUID, IntegerInterval.Set> clean = iter.next();
             UUID cfId = clean.getKey();
-            AtomicInteger cleanPos = clean.getValue();
-            AtomicInteger dirtyPos = cfDirty.get(cfId);
-            if (dirtyPos != null && dirtyPos.intValue() <= cleanPos.intValue())
+            IntegerInterval.Set cleanSet = clean.getValue();
+            IntegerInterval dirtyInterval = cfDirty.get(cfId);
+            if (dirtyInterval != null && cleanSet.covers(dirtyInterval))
             {
                 cfDirty.remove(cfId);
                 iter.remove();
@@ -502,12 +493,12 @@ public abstract class CommitLogSegment
             return cfDirty.keySet();
 
         List<UUID> r = new ArrayList<>(cfDirty.size());
-        for (Map.Entry<UUID, AtomicInteger> dirty : cfDirty.entrySet())
+        for (Map.Entry<UUID, IntegerInterval> dirty : cfDirty.entrySet())
         {
             UUID cfId = dirty.getKey();
-            AtomicInteger dirtyPos = dirty.getValue();
-            AtomicInteger cleanPos = cfClean.get(cfId);
-            if (cleanPos == null || cleanPos.intValue() < dirtyPos.intValue())
+            IntegerInterval dirtyInterval = dirty.getValue();
+            IntegerInterval.Set cleanSet = cfClean.get(cfId);
+            if (cleanSet == null || !cleanSet.covers(dirtyInterval))
                 r.add(dirty.getKey());
         }
         return r;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 66ad6a3..82cee50 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -310,7 +310,7 @@ public class CommitLogSegmentManager
 
             for (CommitLogSegment segment : activeSegments)
                 for (UUID cfId : droppedCfs)
-                    segment.markClean(cfId, segment.getContext());
+                    segment.markClean(cfId, ReplayPosition.NONE, segment.getContext());
 
             // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
             // if the previous active segment was the only one to recycle (since an active segment isn't
@@ -451,7 +451,7 @@ public class CommitLogSegmentManager
                     // even though we remove the schema entry before a final flush when dropping a CF,
                     // it's still possible for a writer to race and finish his append after the flush.
                     logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                    segment.markClean(dirtyCFId, segment.getContext());
+                    segment.markClean(dirtyCFId, ReplayPosition.NONE, segment.getContext());
                 }
                 else if (!flushes.containsKey(dirtyCFId))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
new file mode 100644
index 0000000..bd0ea22
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/commitlog/IntervalSet.java
@@ -0,0 +1,192 @@
+package org.apache.cassandra.db.commitlog;
+
+import java.io.IOException;
+import java.util.*;
+
+import com.google.common.collect.ImmutableSortedMap;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.ISerializer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+/**
+ * An immutable set of closed intervals, stored in normalized form (i.e. where overlapping intervals are converted
+ * to a single interval covering both).
+ *
+ * The set is stored as a sorted map from interval starts to the corresponding end. The map satisfies
+ *   curr().getKey() <= curr().getValue() < next().getKey()
+ */
+public class IntervalSet<T extends Comparable<T>>
+{
+    @SuppressWarnings({ "rawtypes", "unchecked" })
+    private static final IntervalSet EMPTY = new IntervalSet(ImmutableSortedMap.of());
+
+    final private NavigableMap<T, T> ranges;
+
+    private IntervalSet(ImmutableSortedMap<T, T> ranges)
+    {
+        this.ranges = ranges;
+    }
+
+    /**
+     * Construct new set containing the interval with the given start and end position.
+     */
+    public IntervalSet(T start, T end)
+    {
+        this(ImmutableSortedMap.of(start, end));
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <T extends Comparable<T>> IntervalSet<T> empty()
+    {
+        return (IntervalSet<T>) EMPTY;
+    }
+
+    public boolean contains(T position)
+    {
+        // closed (i.e. inclusive) intervals
+        Map.Entry<T, T> range = ranges.floorEntry(position);
+        return range != null && position.compareTo(range.getValue()) <= 0;
+    }
+
+    public boolean isEmpty()
+    {
+        return ranges.isEmpty();
+    }
+
+    public Optional<T> lowerBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.firstKey());
+    }
+
+    public Optional<T> upperBound()
+    {
+        return isEmpty() ? Optional.empty() : Optional.of(ranges.lastEntry().getValue());
+    }
+
+    public Collection<T> starts()
+    {
+        return ranges.keySet();
+    }
+
+    public Collection<T> ends()
+    {
+        return ranges.values();
+    }
+
+    public String toString()
+    {
+        return ranges.toString();
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return ranges.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        return obj instanceof IntervalSet && ranges.equals(((IntervalSet<?>) obj).ranges);
+    }
+
+    public static final <T extends Comparable<T>> ISerializer<IntervalSet<T>> serializer(ISerializer<T> pointSerializer)
+    {
+        return new ISerializer<IntervalSet<T>>()
+        {
+            public void serialize(IntervalSet<T> intervals, DataOutputPlus out) throws IOException
+            {
+                out.writeInt(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    pointSerializer.serialize(en.getKey(), out);
+                    pointSerializer.serialize(en.getValue(), out);
+                }
+            }
+    
+            public IntervalSet<T> deserialize(DataInputPlus in) throws IOException
+            {
+                int count = in.readInt();
+                NavigableMap<T, T> ranges = new TreeMap<>();
+                for (int i = 0; i < count; ++i)
+                    ranges.put(pointSerializer.deserialize(in), pointSerializer.deserialize(in));
+                return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+            }
+    
+            public long serializedSize(IntervalSet<T> intervals)
+            {
+                long size = TypeSizes.sizeof(intervals.ranges.size());
+                for (Map.Entry<T, T> en : intervals.ranges.entrySet())
+                {
+                    size += pointSerializer.serializedSize(en.getKey());
+                    size += pointSerializer.serializedSize(en.getValue());
+                }
+                return size;
+            }
+        };
+    };
+
+    /**
+     * Builder of interval sets, applying the necessary normalization while adding ranges.
+     *
+     * Data is stored as above, as a sorted map from interval starts to the corresponding end, which satisfies
+     *   curr().getKey() <= curr().getValue() < next().getKey()
+     */
+    static public class Builder<T extends Comparable<T>>
+    {
+        final NavigableMap<T, T> ranges;
+
+        public Builder()
+        {
+            this.ranges = new TreeMap<>();
+        }
+
+        public Builder(T start, T end)
+        {
+            this();
+            assert start.compareTo(end) <= 0;
+            ranges.put(start, end);
+        }
+
+        /**
+         * Add an interval to the set and perform normalization.
+         */
+        public void add(T start, T end)
+        {
+            assert start.compareTo(end) <= 0;
+            // extend ourselves to cover any ranges we overlap
+            // record directly preceding our end may extend past us, so take the max of our end and its
+            Map.Entry<T, T> extend = ranges.floorEntry(end);
+            if (extend != null && extend.getValue().compareTo(end) > 0)
+                end = extend.getValue();
+
+            // record directly preceding our start may extend into us; if it does, we take it as our start
+            extend = ranges.lowerEntry(start);
+            if (extend != null && extend.getValue().compareTo(start) >= 0)
+                start = extend.getKey();
+
+            // remove all covered intervals
+            // since we have adjusted start and end to cover the ones that would be only partially covered, we
+            // are certain that anything whose start falls within the span is completely covered
+            ranges.subMap(start, end).clear();
+            // add the new interval
+            ranges.put(start, end);
+        }
+
+        public void addAll(IntervalSet<T> otherSet)
+        {
+            for (Map.Entry<T, T> en : otherSet.ranges.entrySet())
+            {
+                add(en.getKey(), en.getValue());
+            }
+        }
+
+        public IntervalSet<T> build()
+        {
+            return new IntervalSet<T>(ImmutableSortedMap.copyOfSorted(ranges));
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
index 0b21763..b0214b8 100644
--- a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
+++ b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
@@ -18,15 +18,9 @@
 package org.apache.cassandra.db.commitlog;
 
 import java.io.IOException;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-
-import com.google.common.collect.Ordering;
 
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.ISerializer;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 
@@ -43,71 +37,6 @@ public class ReplayPosition implements Comparable<ReplayPosition>
     public final long segment;
     public final int position;
 
-    /**
-     * A filter of known safe-to-discard commit log replay positions, based on
-     * the range covered by on disk sstables and those prior to the most recent truncation record
-     */
-    public static class ReplayFilter
-    {
-        final NavigableMap<ReplayPosition, ReplayPosition> persisted = new TreeMap<>();
-        public ReplayFilter(Iterable<SSTableReader> onDisk, ReplayPosition truncatedAt)
-        {
-            for (SSTableReader reader : onDisk)
-            {
-                ReplayPosition start = reader.getSSTableMetadata().commitLogLowerBound;
-                ReplayPosition end = reader.getSSTableMetadata().commitLogUpperBound;
-                add(persisted, start, end);
-            }
-            if (truncatedAt != null)
-                add(persisted, ReplayPosition.NONE, truncatedAt);
-        }
-
-        private static void add(NavigableMap<ReplayPosition, ReplayPosition> ranges, ReplayPosition start, ReplayPosition end)
-        {
-            // extend ourselves to cover any ranges we overlap
-            // record directly preceding our end may extend past us, so take the max of our end and its
-            Map.Entry<ReplayPosition, ReplayPosition> extend = ranges.floorEntry(end);
-            if (extend != null && extend.getValue().compareTo(end) > 0)
-                end = extend.getValue();
-
-            // record directly preceding our start may extend into us; if it does, we take it as our start
-            extend = ranges.lowerEntry(start);
-            if (extend != null && extend.getValue().compareTo(start) >= 0)
-                start = extend.getKey();
-
-            ranges.subMap(start, end).clear();
-            ranges.put(start, end);
-        }
-
-        public boolean shouldReplay(ReplayPosition position)
-        {
-            // replay ranges are start exclusive, end inclusive
-            Map.Entry<ReplayPosition, ReplayPosition> range = persisted.lowerEntry(position);
-            return range == null || position.compareTo(range.getValue()) > 0;
-        }
-
-        public boolean isEmpty()
-        {
-            return persisted.isEmpty();
-        }
-    }
-
-    public static ReplayPosition firstNotCovered(Iterable<ReplayFilter> ranges)
-    {
-        ReplayPosition min = null;
-        for (ReplayFilter map : ranges)
-        {
-            ReplayPosition first = map.persisted.firstEntry().getValue();
-            if (min == null)
-                min = first;
-            else
-                min = Ordering.natural().min(min, first);
-        }
-        if (min == null)
-            return NONE;
-        return min;
-    }
-
     public ReplayPosition(long segment, int position)
     {
         this.segment = segment;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 0dce52b..a80a6f4 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -241,6 +241,9 @@ public abstract class AbstractCompactionStrategy
      */
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
index 444d43d..a9bfbd2 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
@@ -189,6 +189,9 @@ public class CompactionStrategyManager implements INotificationConsumer
 
     public void replaceFlushed(Memtable memtable, Collection<SSTableReader> sstables)
     {
+        cfs.getTracker().replaceFlushed(memtable, sstables);
+        if (sstables != null && !sstables.isEmpty())
+            CompactionManager.instance.submitBackground(cfs);
     }
 
     public int getUnleveledSSTables()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index c94b88f..5a3d524 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@ -37,7 +37,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
@@ -48,8 +47,6 @@ import org.apache.cassandra.utils.Throwables;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static com.google.common.base.Predicates.and;
-import static com.google.common.base.Predicates.in;
-import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableSet.copyOf;
 import static com.google.common.collect.Iterables.filter;
 import static java.util.Collections.singleton;
@@ -204,7 +201,6 @@ public class Tracker
                          ImmutableList.<Memtable>of(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
                          Collections.<SSTableReader, SSTableReader>emptyMap(),
-                         Collections.<SSTableReader>emptySet(),
                          SSTableIntervalTree.empty()));
     }
 
@@ -351,49 +347,19 @@ public class Tracker
 
         Throwable fail;
         fail = updateSizeTracking(emptySet(), sstables, null);
+        // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+        fail = notifyAdded(sstables, fail);
+
+        if (!isDummy() && !cfstore.isValid())
+            dropSSTables();
 
         maybeFail(fail);
     }
 
-    /**
-     * permit compaction of the provided sstable; this translates to notifying compaction
-     * strategies of its existence, and potentially submitting a background task
-     */
-    public void permitCompactionOfFlushed(Collection<SSTableReader> sstables)
-    {
-        if (sstables.isEmpty())
-            return;
-
-        apply(View.permitCompactionOfFlushed(sstables));
-
-        if (isDummy())
-            return;
-
-        if (cfstore.isValid())
-        {
-            notifyAdded(sstables);
-            CompactionManager.instance.submitBackground(cfstore);
-        }
-        else
-        {
-            dropSSTables();
-        }
-    }
 
 
     // MISCELLANEOUS public utility calls
 
-    public Set<SSTableReader> getSSTables()
-    {
-        return view.get().sstables;
-    }
-
-    public Iterable<SSTableReader> getPermittedToCompact()
-    {
-        View view = this.view.get();
-        return filter(view.sstables, not(in(view.premature)));
-    }
-
     public Set<SSTableReader> getCompacting()
     {
         return view.get().compacting;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index 3fa197f..4b3aae0 100644
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@ -33,7 +33,6 @@ 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.in;
 import static com.google.common.base.Predicates.not;
 import static com.google.common.collect.ImmutableList.copyOf;
 import static com.google.common.collect.ImmutableList.of;
@@ -70,7 +69,6 @@ public class View
     public final List<Memtable> flushingMemtables;
     final Set<SSTableReader> compacting;
     final Set<SSTableReader> sstables;
-    final Set<SSTableReader> premature;
     // 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
@@ -80,7 +78,7 @@ public class View
 
     final SSTableIntervalTree intervalTree;
 
-    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, Set<SSTableReader> premature, SSTableIntervalTree intervalTree)
+    View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Map<SSTableReader, SSTableReader> sstables, Map<SSTableReader, SSTableReader> compacting, SSTableIntervalTree intervalTree)
     {
         assert liveMemtables != null;
         assert flushingMemtables != null;
@@ -95,7 +93,6 @@ public class View
         this.sstables = sstablesMap.keySet();
         this.compactingMap = compacting;
         this.compacting = compactingMap.keySet();
-        this.premature = premature;
         this.intervalTree = intervalTree;
     }
 
@@ -256,7 +253,7 @@ public class View
                 assert all(mark, Helpers.idIn(view.sstablesMap));
                 return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap,
                                 replace(view.compactingMap, unmark, mark),
-                                view.premature, view.intervalTree);
+                                view.intervalTree);
             }
         };
     }
@@ -270,7 +267,7 @@ public class View
             public boolean apply(View view)
             {
                 for (SSTableReader reader : readers)
-                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted() || view.premature.contains(reader))
+                    if (view.compacting.contains(reader) || view.sstablesMap.get(reader) != reader || reader.isMarkedCompacted())
                         return false;
                 return true;
             }
@@ -287,7 +284,7 @@ public class View
             public View apply(View view)
             {
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, remove, add);
-                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap, view.premature,
+                return new View(view.liveMemtables, view.flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
@@ -302,7 +299,7 @@ public class 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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, view.flushingMemtables, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -321,7 +318,7 @@ public class View
                                                            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.compactingMap, view.premature, view.intervalTree);
+                return new View(newLive, newFlushing, view.sstablesMap, view.compactingMap, view.intervalTree);
             }
         };
     }
@@ -338,32 +335,15 @@ public class View
 
                 if (flushed == null || flushed.isEmpty())
                     return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
-                                    view.compactingMap, view.premature, view.intervalTree);
+                                    view.compactingMap, view.intervalTree);
 
                 Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed);
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, emptySet(), flushed);
-                Set<SSTableReader> premature = replace(view.premature, emptySet(), flushed);
-                return new View(view.liveMemtables, flushingMemtables, sstableMap, compactingMap, premature,
+                return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap,
                                 SSTableIntervalTree.build(sstableMap.keySet()));
             }
         };
     }
 
-    static Function<View, View> permitCompactionOfFlushed(final Collection<SSTableReader> readers)
-    {
-        Set<SSTableReader> expectAndRemove = ImmutableSet.copyOf(readers);
-        return new Function<View, View>()
-        {
-            public View apply(View view)
-            {
-                Set<SSTableReader> premature = replace(view.premature, expectAndRemove, emptySet());
-                Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, expectAndRemove, emptySet());
-                return new View(view.liveMemtables, view.flushingMemtables, view.sstablesMap, compactingMap, premature, view.intervalTree);
-            }
-        };
-    }
-
-
     private static <T extends Comparable<T>> Predicate<T> lessThan(final T lessThan)
     {
         return new Predicate<T>()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/Version.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/Version.java b/src/java/org/apache/cassandra/io/sstable/format/Version.java
index d9e289c..96c5a6e 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/Version.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/Version.java
@@ -72,6 +72,8 @@ public abstract class Version
 
     public abstract boolean hasCommitLogLowerBound();
 
+    public abstract boolean hasCommitLogIntervals();
+
     public String getVersion()
     {
         return version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
index e0fb3b1..16f0beb 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
@@ -111,7 +111,7 @@ public class BigFormat implements SSTableFormat
     // we always incremented the major version.
     static class BigVersion extends Version
     {
-        public static final String current_version = "mb";
+        public static final String current_version = "mc";
         public static final String earliest_supported_version = "jb";
 
         // jb (2.0.1): switch from crc32 to adler32 for compression checksums
@@ -124,7 +124,8 @@ public class BigFormat implements SSTableFormat
         // lb (2.2.7): commit log lower bound included
         // ma (3.0.0): swap bf hash order
         //             store rows natively
-        // mb (3.0.6): commit log lower bound included
+        // mb (3.0.7, 3.7): commit log lower bound included
+        // mc (3.0.8, 3.9): commit log intervals included
         //
         // NOTE: when adding a new version, please add that to LegacySSTableTest, too.
 
@@ -145,6 +146,7 @@ public class BigFormat implements SSTableFormat
          */
         private final boolean hasOldBfHashOrder;
         private final boolean hasCommitLogLowerBound;
+        private final boolean hasCommitLogIntervals;
 
         /**
          * CASSANDRA-7066: compaction ancerstors are no longer used and have been removed.
@@ -186,6 +188,7 @@ public class BigFormat implements SSTableFormat
             hasBoundaries = version.compareTo("ma") < 0;
             hasCommitLogLowerBound = (version.compareTo("lb") >= 0 && version.compareTo("ma") < 0)
                                      || version.compareTo("mb") >= 0;
+            hasCommitLogIntervals = version.compareTo("mc") >= 0;
         }
 
         @Override
@@ -248,12 +251,19 @@ public class BigFormat implements SSTableFormat
             return newFileName;
         }
 
+        @Override
         public boolean hasCommitLogLowerBound()
         {
             return hasCommitLogLowerBound;
         }
 
         @Override
+        public boolean hasCommitLogIntervals()
+        {
+            return hasCommitLogIntervals;
+        }
+
+        @Override
         public boolean storeRows()
         {
             return storeRows;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 4561520..a683513 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@ -24,6 +24,7 @@ import java.util.*;
 import com.google.common.collect.Maps;
 
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
@@ -35,6 +36,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.StreamingHistogram;
 
+import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.replayPositionSetSerializer;
+
 /**
  * Serializer for SSTable from legacy versions
  */
@@ -55,7 +58,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
         EstimatedHistogram.serializer.serialize(stats.estimatedPartitionSize, out);
         EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-        ReplayPosition.serializer.serialize(stats.commitLogUpperBound, out);
+        ReplayPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
         out.writeLong(stats.minTimestamp);
         out.writeLong(stats.maxTimestamp);
         out.writeInt(stats.maxLocalDeletionTime);
@@ -72,7 +75,9 @@ public class LegacyMetadataSerializer extends MetadataSerializer
         for (ByteBuffer value : stats.maxClusteringValues)
             ByteBufferUtil.writeWithShortLength(value, out);
         if (version.hasCommitLogLowerBound())
-            ReplayPosition.serializer.serialize(stats.commitLogLowerBound, out);
+            ReplayPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+        if (version.hasCommitLogIntervals())
+            replayPositionSetSerializer.serialize(stats.commitLogIntervals, out);
     }
 
     /**
@@ -121,6 +126,11 @@ public class LegacyMetadataSerializer extends MetadataSerializer
 
                 if (descriptor.version.hasCommitLogLowerBound())
                     commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+                IntervalSet<ReplayPosition> commitLogIntervals;
+                if (descriptor.version.hasCommitLogIntervals())
+                    commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+                else
+                    commitLogIntervals = new IntervalSet<>(commitLogLowerBound, commitLogUpperBound);
 
                 if (types.contains(MetadataType.VALIDATION))
                     components.put(MetadataType.VALIDATION,
@@ -129,8 +139,7 @@ public class LegacyMetadataSerializer extends MetadataSerializer
                     components.put(MetadataType.STATS,
                                    new StatsMetadata(partitionSizes,
                                                      columnCounts,
-                                                     commitLogLowerBound,
-                                                     commitLogUpperBound,
+                                                     commitLogIntervals,
                                                      minTimestamp,
                                                      maxTimestamp,
                                                      Integer.MAX_VALUE,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 53cf0b0..1ff2ca8 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -27,12 +27,11 @@ import java.util.Map;
 import java.util.Set;
 
 import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
 
 import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
 import com.clearspring.analytics.stream.cardinality.ICardinality;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
 import org.apache.cassandra.db.rows.Cell;
@@ -69,8 +68,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         return new StatsMetadata(defaultPartitionSizeHistogram(),
                                  defaultCellPerPartitionCountHistogram(),
-                                 ReplayPosition.NONE,
-                                 ReplayPosition.NONE,
+                                 IntervalSet.empty(),
                                  Long.MIN_VALUE,
                                  Long.MAX_VALUE,
                                  Integer.MAX_VALUE,
@@ -91,8 +89,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
     protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
     // TODO: cound the number of row per partition (either with the number of cells, or instead)
     protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
-    protected ReplayPosition commitLogLowerBound = ReplayPosition.NONE;
-    protected ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
+    protected IntervalSet commitLogIntervals = IntervalSet.empty();
     protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
     protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
     protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
@@ -126,23 +123,13 @@ public class MetadataCollector implements PartitionStatisticsCollector
     {
         this(comparator);
 
-        ReplayPosition min = null, max = null;
+        IntervalSet.Builder intervals = new IntervalSet.Builder();
         for (SSTableReader sstable : sstables)
         {
-            if (min == null)
-            {
-                min = sstable.getSSTableMetadata().commitLogLowerBound;
-                max = sstable.getSSTableMetadata().commitLogUpperBound;
-            }
-            else
-            {
-                min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
-                max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
-            }
+            intervals.addAll(sstable.getSSTableMetadata().commitLogIntervals);
         }
 
-        commitLogLowerBound(min);
-        commitLogUpperBound(max);
+        commitLogIntervals(intervals.build());
         sstableLevel(level);
     }
 
@@ -229,15 +216,9 @@ public class MetadataCollector implements PartitionStatisticsCollector
         ttlTracker.update(newTTL);
     }
 
-    public MetadataCollector commitLogLowerBound(ReplayPosition commitLogLowerBound)
-    {
-        this.commitLogLowerBound = commitLogLowerBound;
-        return this;
-    }
-
-    public MetadataCollector commitLogUpperBound(ReplayPosition commitLogUpperBound)
+    public MetadataCollector commitLogIntervals(IntervalSet commitLogIntervals)
     {
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         return this;
     }
 
@@ -302,8 +283,7 @@ public class MetadataCollector implements PartitionStatisticsCollector
         components.put(MetadataType.VALIDATION, new ValidationMetadata(partitioner, bloomFilterFPChance));
         components.put(MetadataType.STATS, new StatsMetadata(estimatedPartitionSize,
                                                              estimatedCellPerPartitionCount,
-                                                             commitLogLowerBound,
-                                                             commitLogUpperBound,
+                                                             commitLogIntervals,
                                                              timestampTracker.min(),
                                                              timestampTracker.max(),
                                                              localDeletionTimeTracker.min(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index 07e35bb..9971eaa 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@ -22,10 +22,12 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.commons.lang3.builder.EqualsBuilder;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -39,11 +41,11 @@ import org.apache.cassandra.utils.StreamingHistogram;
 public class StatsMetadata extends MetadataComponent
 {
     public static final IMetadataComponentSerializer serializer = new StatsMetadataSerializer();
+    public static final ISerializer<IntervalSet<ReplayPosition>> replayPositionSetSerializer = IntervalSet.serializer(ReplayPosition.serializer);
 
     public final EstimatedHistogram estimatedPartitionSize;
     public final EstimatedHistogram estimatedColumnCount;
-    public final ReplayPosition commitLogLowerBound;
-    public final ReplayPosition commitLogUpperBound;
+    public final IntervalSet<ReplayPosition> commitLogIntervals;
     public final long minTimestamp;
     public final long maxTimestamp;
     public final int minLocalDeletionTime;
@@ -62,8 +64,7 @@ public class StatsMetadata extends MetadataComponent
 
     public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                          EstimatedHistogram estimatedColumnCount,
-                         ReplayPosition commitLogLowerBound,
-                         ReplayPosition commitLogUpperBound,
+                         IntervalSet<ReplayPosition> commitLogIntervals,
                          long minTimestamp,
                          long maxTimestamp,
                          int minLocalDeletionTime,
@@ -82,8 +83,7 @@ public class StatsMetadata extends MetadataComponent
     {
         this.estimatedPartitionSize = estimatedPartitionSize;
         this.estimatedColumnCount = estimatedColumnCount;
-        this.commitLogLowerBound = commitLogLowerBound;
-        this.commitLogUpperBound = commitLogUpperBound;
+        this.commitLogIntervals = commitLogIntervals;
         this.minTimestamp = minTimestamp;
         this.maxTimestamp = maxTimestamp;
         this.minLocalDeletionTime = minLocalDeletionTime;
@@ -134,8 +134,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -157,8 +156,7 @@ public class StatsMetadata extends MetadataComponent
     {
         return new StatsMetadata(estimatedPartitionSize,
                                  estimatedColumnCount,
-                                 commitLogLowerBound,
-                                 commitLogUpperBound,
+                                 commitLogIntervals,
                                  minTimestamp,
                                  maxTimestamp,
                                  minLocalDeletionTime,
@@ -186,8 +184,7 @@ public class StatsMetadata extends MetadataComponent
         return new EqualsBuilder()
                        .append(estimatedPartitionSize, that.estimatedPartitionSize)
                        .append(estimatedColumnCount, that.estimatedColumnCount)
-                       .append(commitLogLowerBound, that.commitLogLowerBound)
-                       .append(commitLogUpperBound, that.commitLogUpperBound)
+                       .append(commitLogIntervals, that.commitLogIntervals)
                        .append(minTimestamp, that.minTimestamp)
                        .append(maxTimestamp, that.maxTimestamp)
                        .append(minLocalDeletionTime, that.minLocalDeletionTime)
@@ -212,8 +209,7 @@ public class StatsMetadata extends MetadataComponent
         return new HashCodeBuilder()
                        .append(estimatedPartitionSize)
                        .append(estimatedColumnCount)
-                       .append(commitLogLowerBound)
-                       .append(commitLogUpperBound)
+                       .append(commitLogIntervals)
                        .append(minTimestamp)
                        .append(maxTimestamp)
                        .append(minLocalDeletionTime)
@@ -239,7 +235,7 @@ public class StatsMetadata extends MetadataComponent
             int size = 0;
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize);
             size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount);
-            size += ReplayPosition.serializer.serializedSize(component.commitLogUpperBound);
+            size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE));
             if (version.storeRows())
                 size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long)
             else
@@ -258,7 +254,9 @@ public class StatsMetadata extends MetadataComponent
             if (version.storeRows())
                 size += 8 + 8; // totalColumnsSet, totalRows
             if (version.hasCommitLogLowerBound())
-                size += ReplayPosition.serializer.serializedSize(component.commitLogLowerBound);
+                size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE));
+            if (version.hasCommitLogIntervals())
+                size += replayPositionSetSerializer.serializedSize(component.commitLogIntervals);
             return size;
         }
 
@@ -266,7 +264,7 @@ public class StatsMetadata extends MetadataComponent
         {
             EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out);
             EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-            ReplayPosition.serializer.serialize(component.commitLogUpperBound, out);
+            ReplayPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
             out.writeLong(component.minTimestamp);
             out.writeLong(component.maxTimestamp);
             if (version.storeRows())
@@ -296,7 +294,9 @@ public class StatsMetadata extends MetadataComponent
             }
 
             if (version.hasCommitLogLowerBound())
-                ReplayPosition.serializer.serialize(component.commitLogLowerBound, out);
+                ReplayPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
+            if (version.hasCommitLogIntervals())
+                replayPositionSetSerializer.serialize(component.commitLogIntervals, out);
         }
 
         public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@ -338,11 +338,15 @@ public class StatsMetadata extends MetadataComponent
 
             if (version.hasCommitLogLowerBound())
                 commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
+            IntervalSet<ReplayPosition> commitLogIntervals;
+            if (version.hasCommitLogIntervals())
+                commitLogIntervals = replayPositionSetSerializer.deserialize(in);
+            else
+                commitLogIntervals = new IntervalSet<ReplayPosition>(commitLogLowerBound, commitLogUpperBound);
 
             return new StatsMetadata(partitionSizes,
                                      columnCounts,
-                                     commitLogLowerBound,
-                                     commitLogUpperBound,
+                                     commitLogIntervals,
                                      minTimestamp,
                                      maxTimestamp,
                                      minLocalDeletionTime,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 420b802..5f7513f 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -70,8 +70,7 @@ public class SSTableMetadataViewer
                     out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                     out.printf("SSTable Level: %d%n", stats.sstableLevel);
                     out.printf("Repaired at: %d%n", stats.repairedAt);
-                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
-                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                     out.println("Estimated tombstone drop times:");
                     for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                     {


[19/23] cassandra git commit: Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index b1c706e,5a3d524..f464e08
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -353,35 -347,13 +349,16 @@@ public class Tracke
  
          Throwable fail;
          fail = updateSizeTracking(emptySet(), sstables, null);
 +
 +        notifyDiscarded(memtable);
 +
-         maybeFail(fail);
-     }
- 
-     /**
-      * permit compaction of the provided sstable; this translates to notifying compaction
-      * strategies of its existence, and potentially submitting a background task
-      */
-     public void permitCompactionOfFlushed(Collection<SSTableReader> sstables)
-     {
-         if (sstables.isEmpty())
-             return;
+         // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+         fail = notifyAdded(sstables, fail);
  
-         apply(View.permitCompactionOfFlushed(sstables));
- 
-         if (isDummy())
-             return;
- 
-         if (cfstore.isValid())
-         {
-             notifyAdded(sstables);
-             CompactionManager.instance.submitBackground(cfstore);
-         }
-         else
-         {
+         if (!isDummy() && !cfstore.isValid())
              dropSSTables();
-         }
+ 
+         maybeFail(fail);
      }
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/View.java
index a5c781d,4b3aae0..b26426d
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@@ -40,7 -39,7 +39,6 @@@ import static com.google.common.collect
  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 com.google.common.collect.Iterables.transform;
  import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
  import static org.apache.cassandra.db.lifecycle.Helpers.filterOut;
  import static org.apache.cassandra.db.lifecycle.Helpers.replace;
@@@ -336,14 -333,12 +332,12 @@@ public class Vie
                  List<Memtable> flushingMemtables = copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
                  assert flushingMemtables.size() == view.flushingMemtables.size() - 1;
  
 -                if (flushed == null || flushed.isEmpty())
 +                if (flushed == null || Iterables.isEmpty(flushed))
                      return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
-                                     view.compactingMap, view.premature, view.intervalTree);
+                                     view.compactingMap, view.intervalTree);
  
                  Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed);
-                 Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, emptySet(), flushed);
-                 Set<SSTableReader> premature = replace(view.premature, emptySet(), flushed);
-                 return new View(view.liveMemtables, flushingMemtables, sstableMap, compactingMap, premature,
+                 return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap,
                                  SSTableIntervalTree.build(sstableMap.keySet()));
              }
          };

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 505de49,a683513..14e391b
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@@ -24,7 -24,8 +24,8 @@@ import java.util.*
  import com.google.common.collect.Maps;
  
  import org.apache.cassandra.db.TypeSizes;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.format.Version;
@@@ -35,6 -36,8 +36,8 @@@ import org.apache.cassandra.utils.ByteB
  import org.apache.cassandra.utils.EstimatedHistogram;
  import org.apache.cassandra.utils.StreamingHistogram;
  
 -import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.replayPositionSetSerializer;
++import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.commitLogPositionSetSerializer;
+ 
  /**
   * Serializer for SSTable from legacy versions
   */
@@@ -55,7 -58,7 +58,7 @@@ public class LegacyMetadataSerializer e
  
          EstimatedHistogram.serializer.serialize(stats.estimatedPartitionSize, out);
          EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-         CommitLogPosition.serializer.serialize(stats.commitLogUpperBound, out);
 -        ReplayPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
++        CommitLogPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE), out);
          out.writeLong(stats.minTimestamp);
          out.writeLong(stats.maxTimestamp);
          out.writeInt(stats.maxLocalDeletionTime);
@@@ -72,7 -75,9 +75,9 @@@
          for (ByteBuffer value : stats.maxClusteringValues)
              ByteBufferUtil.writeWithShortLength(value, out);
          if (version.hasCommitLogLowerBound())
-             CommitLogPosition.serializer.serialize(stats.commitLogLowerBound, out);
 -            ReplayPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
++            CommitLogPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE), out);
+         if (version.hasCommitLogIntervals())
 -            replayPositionSetSerializer.serialize(stats.commitLogIntervals, out);
++            commitLogPositionSetSerializer.serialize(stats.commitLogIntervals, out);
      }
  
      /**
@@@ -120,7 -125,12 +125,12 @@@
                      maxColumnNames.add(ByteBufferUtil.readWithShortLength(in));
  
                  if (descriptor.version.hasCommitLogLowerBound())
 -                    commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
 -                IntervalSet<ReplayPosition> commitLogIntervals;
 +                    commitLogLowerBound = CommitLogPosition.serializer.deserialize(in);
++                IntervalSet<CommitLogPosition> commitLogIntervals;
+                 if (descriptor.version.hasCommitLogIntervals())
 -                    commitLogIntervals = replayPositionSetSerializer.deserialize(in);
++                    commitLogIntervals = commitLogPositionSetSerializer.deserialize(in);
+                 else
+                     commitLogIntervals = new IntervalSet<>(commitLogLowerBound, commitLogUpperBound);
  
                  if (types.contains(MetadataType.VALIDATION))
                      components.put(MetadataType.VALIDATION,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 299bc87,1ff2ca8..196cfbf
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@@ -20,16 -20,18 +20,16 @@@ package org.apache.cassandra.io.sstable
  import java.nio.ByteBuffer;
  import java.util.ArrayList;
  import java.util.Collections;
 -import java.util.HashSet;
  import java.util.List;
  import java.util.Map;
 -import java.util.Set;
  
  import com.google.common.collect.Maps;
- import com.google.common.collect.Ordering;
  
  import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
  import com.clearspring.analytics.stream.cardinality.ICardinality;
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
  import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
  import org.apache.cassandra.db.rows.Cell;
@@@ -88,8 -89,7 +87,7 @@@ public class MetadataCollector implemen
      protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
      // TODO: cound the number of row per partition (either with the number of cells, or instead)
      protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
-     protected CommitLogPosition commitLogLowerBound = CommitLogPosition.NONE;
-     protected CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE;
 -    protected IntervalSet commitLogIntervals = IntervalSet.empty();
++    protected IntervalSet<CommitLogPosition> commitLogIntervals = IntervalSet.empty();
      protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
      protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
      protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
@@@ -123,23 -123,13 +121,13 @@@
      {
          this(comparator);
  
-         CommitLogPosition min = null, max = null;
 -        IntervalSet.Builder intervals = new IntervalSet.Builder();
++        IntervalSet.Builder<CommitLogPosition> intervals = new IntervalSet.Builder<>();
          for (SSTableReader sstable : sstables)
          {
-             if (min == null)
-             {
-                 min = sstable.getSSTableMetadata().commitLogLowerBound;
-                 max = sstable.getSSTableMetadata().commitLogUpperBound;
-             }
-             else
-             {
-                 min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
-                 max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
-             }
+             intervals.addAll(sstable.getSSTableMetadata().commitLogIntervals);
          }
  
-         commitLogLowerBound(min);
-         commitLogUpperBound(max);
+         commitLogIntervals(intervals.build());
          sstableLevel(level);
      }
  
@@@ -226,15 -216,9 +214,9 @@@
          ttlTracker.update(newTTL);
      }
  
-     public MetadataCollector commitLogLowerBound(CommitLogPosition commitLogLowerBound)
 -    public MetadataCollector commitLogIntervals(IntervalSet commitLogIntervals)
++    public MetadataCollector commitLogIntervals(IntervalSet<CommitLogPosition> commitLogIntervals)
      {
-         this.commitLogLowerBound = commitLogLowerBound;
-         return this;
-     }
- 
-     public MetadataCollector commitLogUpperBound(CommitLogPosition commitLogUpperBound)
-     {
-         this.commitLogUpperBound = commitLogUpperBound;
+         this.commitLogIntervals = commitLogIntervals;
          return this;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index e765235,9971eaa..c83c2cf
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@@ -26,7 -27,8 +27,8 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.commons.lang3.builder.EqualsBuilder;
  import org.apache.commons.lang3.builder.HashCodeBuilder;
  import org.apache.cassandra.db.TypeSizes;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.io.util.DataInputPlus;
  import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.utils.ByteBufferUtil;
@@@ -39,11 -41,11 +41,11 @@@ import org.apache.cassandra.utils.Strea
  public class StatsMetadata extends MetadataComponent
  {
      public static final IMetadataComponentSerializer serializer = new StatsMetadataSerializer();
 -    public static final ISerializer<IntervalSet<ReplayPosition>> replayPositionSetSerializer = IntervalSet.serializer(ReplayPosition.serializer);
++    public static final ISerializer<IntervalSet<CommitLogPosition>> commitLogPositionSetSerializer = IntervalSet.serializer(CommitLogPosition.serializer);
  
      public final EstimatedHistogram estimatedPartitionSize;
      public final EstimatedHistogram estimatedColumnCount;
-     public final CommitLogPosition commitLogLowerBound;
-     public final CommitLogPosition commitLogUpperBound;
 -    public final IntervalSet<ReplayPosition> commitLogIntervals;
++    public final IntervalSet<CommitLogPosition> commitLogIntervals;
      public final long minTimestamp;
      public final long maxTimestamp;
      public final int minLocalDeletionTime;
@@@ -62,8 -64,7 +64,7 @@@
  
      public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                           EstimatedHistogram estimatedColumnCount,
-                          CommitLogPosition commitLogLowerBound,
-                          CommitLogPosition commitLogUpperBound,
 -                         IntervalSet<ReplayPosition> commitLogIntervals,
++                         IntervalSet<CommitLogPosition> commitLogIntervals,
                           long minTimestamp,
                           long maxTimestamp,
                           int minLocalDeletionTime,
@@@ -239,7 -235,7 +235,7 @@@
              int size = 0;
              size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize);
              size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount);
-             size += CommitLogPosition.serializer.serializedSize(component.commitLogUpperBound);
 -            size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE));
++            size += CommitLogPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE));
              if (version.storeRows())
                  size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long)
              else
@@@ -258,7 -254,9 +254,9 @@@
              if (version.storeRows())
                  size += 8 + 8; // totalColumnsSet, totalRows
              if (version.hasCommitLogLowerBound())
-                 size += CommitLogPosition.serializer.serializedSize(component.commitLogLowerBound);
 -                size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE));
++                size += CommitLogPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE));
+             if (version.hasCommitLogIntervals())
 -                size += replayPositionSetSerializer.serializedSize(component.commitLogIntervals);
++                size += commitLogPositionSetSerializer.serializedSize(component.commitLogIntervals);
              return size;
          }
  
@@@ -266,7 -264,7 +264,7 @@@
          {
              EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out);
              EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-             CommitLogPosition.serializer.serialize(component.commitLogUpperBound, out);
 -            ReplayPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
++            CommitLogPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE), out);
              out.writeLong(component.minTimestamp);
              out.writeLong(component.maxTimestamp);
              if (version.storeRows())
@@@ -296,7 -294,9 +294,9 @@@
              }
  
              if (version.hasCommitLogLowerBound())
-                 CommitLogPosition.serializer.serialize(component.commitLogLowerBound, out);
 -                ReplayPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
++                CommitLogPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE), out);
+             if (version.hasCommitLogIntervals())
 -                replayPositionSetSerializer.serialize(component.commitLogIntervals, out);
++                commitLogPositionSetSerializer.serialize(component.commitLogIntervals, out);
          }
  
          public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@@ -337,7 -337,12 +337,12 @@@
              long totalRows = version.storeRows() ? in.readLong() : -1L;
  
              if (version.hasCommitLogLowerBound())
 -                commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
 -            IntervalSet<ReplayPosition> commitLogIntervals;
 +                commitLogLowerBound = CommitLogPosition.serializer.deserialize(in);
++            IntervalSet<CommitLogPosition> commitLogIntervals;
+             if (version.hasCommitLogIntervals())
 -                commitLogIntervals = replayPositionSetSerializer.deserialize(in);
++                commitLogIntervals = commitLogPositionSetSerializer.deserialize(in);
+             else
 -                commitLogIntervals = new IntervalSet<ReplayPosition>(commitLogLowerBound, commitLogUpperBound);
++                commitLogIntervals = new IntervalSet<CommitLogPosition>(commitLogLowerBound, commitLogUpperBound);
  
              return new StatsMetadata(partitionSizes,
                                       columnCounts,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 3c8ba64,5f7513f..6686684
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -112,15 -70,11 +112,14 @@@ public class SSTableMetadataViewe
                      out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
-                     out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
-                     out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                     out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
 +                    out.printf("totalColumnsSet: %s%n", stats.totalColumnsSet);
 +                    out.printf("totalRows: %s%n", stats.totalRows);
                      out.println("Estimated tombstone drop times:");
 -                    for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
 +
 +                    for (Map.Entry<Number, long[]> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                      {
 -                        out.printf("%-10s:%10s%n",entry.getKey().intValue(), entry.getValue());
 +                        out.printf("%-10s:%10s%n",entry.getKey().intValue(), entry.getValue()[0]);
                      }
                      printHistograms(stats, out);
                  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --cc test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 239077e,02b26c7..2858597
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@@ -250,9 -245,9 +250,10 @@@ public class CommitLogStressTes
              }
              verifySizes(commitLog);
  
-             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId, discardedPos);
+             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
 -                    ReplayPosition.NONE, discardedPos);
++                    CommitLogPosition.NONE, discardedPos);
              threads.clear();
 +
              System.out.format("Discarded at %s\n", discardedPos);
              verifySizes(commitLog);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/Util.java
index cd709d5,d04ca9b..7bcee7a
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@@ -677,9 -648,26 +679,26 @@@ public class Uti
          }
  
          private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
 -                                                                 ReadOrderGroup orderGroup)
 +                                                                 ReadExecutionController controller)
          {
 -            return queryStorage(cfs, orderGroup);
 +            return queryStorage(cfs, controller);
          }
      }
+ 
+     public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)
+     {
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+         return () -> BlacklistedDirectories.clearUnwritableUnsafe();
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 23ec58b,9a0ddb8..6ab7d46
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@@ -20,9 -20,13 +20,10 @@@ package org.apache.cassandra.db.commitl
  
  import java.io.*;
  import java.nio.ByteBuffer;
 -import java.util.Arrays;
 -import java.util.Collection;
 -import java.util.Collections;
 -import java.util.UUID;
 +import java.util.*;
  import java.util.concurrent.Callable;
  import java.util.concurrent.ExecutionException;
+ import java.util.function.BiConsumer;
  import java.util.zip.CRC32;
  import java.util.zip.Checksum;
  
@@@ -35,34 -40,27 +36,34 @@@ import org.junit.runners.Parameterized.
  
  import org.apache.cassandra.SchemaLoader;
  import org.apache.cassandra.Util;
 +import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.ParameterizedClass;
- import org.apache.cassandra.db.ColumnFamilyStore;
- import org.apache.cassandra.db.Keyspace;
- import org.apache.cassandra.db.Mutation;
- import org.apache.cassandra.db.RowUpdateBuilder;
+ import org.apache.cassandra.config.Config.DiskFailurePolicy;
+ import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.marshal.AsciiType;
  import org.apache.cassandra.db.marshal.BytesType;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.Row;
  import org.apache.cassandra.exceptions.ConfigurationException;
+ import org.apache.cassandra.io.FSWriteError;
  import org.apache.cassandra.io.compress.DeflateCompressor;
  import org.apache.cassandra.io.compress.LZ4Compressor;
  import org.apache.cassandra.io.compress.SnappyCompressor;
+ import org.apache.cassandra.io.sstable.format.SSTableReader;
  import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.schema.KeyspaceParams;
 -import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.security.EncryptionContext;
 +import org.apache.cassandra.security.EncryptionContextGenerator;
 +import org.apache.cassandra.utils.Hex;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.KillerForTests;
 +import org.apache.cassandra.utils.Pair;
  import org.apache.cassandra.utils.vint.VIntCoding;
  
 +import org.junit.After;
  import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertTrue;
@@@ -247,13 -228,13 +248,13 @@@ public class CommitLogTes
                        .build();
          CommitLog.instance.add(m2);
  
 -        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 +        assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
  
          UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-         CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
 -        // Assert we still have both our segment
 -        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 +        // Assert we still have both our segments
 +        assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
      }
  
      @Test
@@@ -278,9 -258,9 +279,9 @@@
          // "Flush": this won't delete anything
          UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
          CommitLog.instance.sync(true);
-         CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid1, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 +        assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
  
          // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
          Mutation rm2 = new RowUpdateBuilder(cfs2.metadata, 0, "k")
@@@ -298,10 -279,10 +299,10 @@@
          // didn't write anything on cf1 since last flush (and we flush cf2)
  
          UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-         CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
          // Assert we still have both our segment
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 +        assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
      }
  
      private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String cfName, String colName)
@@@ -545,13 -486,13 +546,13 @@@
              for (int i = 0 ; i < 5 ; i++)
                  CommitLog.instance.add(m2);
  
 -            assertEquals(2, CommitLog.instance.activeSegments());
 -            ReplayPosition position = CommitLog.instance.getContext();
 -            for (Keyspace ks : Keyspace.system())
 -                for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
 -                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, ReplayPosition.NONE, position);
 -            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, ReplayPosition.NONE, position);
 -            assertEquals(1, CommitLog.instance.activeSegments());
 +            assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
 +            CommitLogPosition position = CommitLog.instance.getCurrentPosition();
 +            for (Keyspace keyspace : Keyspace.system())
 +                for (ColumnFamilyStore syscfs : keyspace.getColumnFamilyStores())
-                     CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-             CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
++                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, CommitLogPosition.NONE, position);
++            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, CommitLogPosition.NONE, position);
 +            assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
          }
          finally
          {
@@@ -589,108 -530,136 +590,240 @@@
      }
  
      @Test
 +    public void replaySimple() throws IOException
 +    {
 +        int cellCount = 0;
 +        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 +        final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k1")
 +                             .clustering("bytes")
 +                             .add("val", bytes("this is a string"))
 +                             .build();
 +        cellCount += 1;
 +        CommitLog.instance.add(rm1);
 +
 +        final Mutation rm2 = new RowUpdateBuilder(cfs.metadata, 0, "k2")
 +                             .clustering("bytes")
 +                             .add("val", bytes("this is a string"))
 +                             .build();
 +        cellCount += 1;
 +        CommitLog.instance.add(rm2);
 +
 +        CommitLog.instance.sync(true);
 +
 +        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, CommitLogPosition.NONE, cfs.metadata);
 +        List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
 +        Assert.assertFalse(activeSegments.isEmpty());
 +
 +        File[] files = new File(CommitLog.instance.segmentManager.storageDirectory).listFiles((file, name) -> activeSegments.contains(name));
 +        replayer.replayFiles(files);
 +
 +        assertEquals(cellCount, replayer.cells);
 +    }
 +
 +    @Test
 +    public void replayWithDiscard() throws IOException
 +    {
 +        int cellCount = 0;
 +        int max = 1024;
 +        int discardPosition = (int)(max * .8); // an arbitrary number of entries that we'll skip on the replay
 +        CommitLogPosition commitLogPosition = null;
 +        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 +
 +        for (int i = 0; i < max; i++)
 +        {
 +            final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k" + 1)
 +                                 .clustering("bytes")
 +                                 .add("val", bytes("this is a string"))
 +                                 .build();
 +            CommitLogPosition position = CommitLog.instance.add(rm1);
 +
 +            if (i == discardPosition)
 +                commitLogPosition = position;
 +            if (i > discardPosition)
 +            {
 +                cellCount += 1;
 +            }
 +        }
 +
 +        CommitLog.instance.sync(true);
 +
 +        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, commitLogPosition, cfs.metadata);
 +        List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
 +        Assert.assertFalse(activeSegments.isEmpty());
 +
 +        File[] files = new File(CommitLog.instance.segmentManager.storageDirectory).listFiles((file, name) -> activeSegments.contains(name));
 +        replayer.replayFiles(files);
 +
 +        assertEquals(cellCount, replayer.cells);
 +    }
 +
 +    class SimpleCountingReplayer extends CommitLogReplayer
 +    {
 +        private final CommitLogPosition filterPosition;
 +        private final CFMetaData metadata;
 +        int cells;
 +        int skipped;
 +
 +        SimpleCountingReplayer(CommitLog commitLog, CommitLogPosition filterPosition, CFMetaData cfm)
 +        {
 +            super(commitLog, filterPosition, Collections.emptyMap(), ReplayFilter.create());
 +            this.filterPosition = filterPosition;
 +            this.metadata = cfm;
 +        }
 +
 +        @SuppressWarnings("resource")
 +        @Override
 +        public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc)
 +        {
 +            if (entryLocation <= filterPosition.position)
 +            {
 +                // Skip over this mutation.
 +                skipped++;
 +                return;
 +            }
 +            for (PartitionUpdate partitionUpdate : m.getPartitionUpdates())
 +            {
 +                // Only process mutations for the CF's we're testing against, since we can't deterministically predict
 +                // whether or not system keyspaces will be mutated during a test.
 +                if (partitionUpdate.metadata().cfName.equals(metadata.cfName))
 +                {
 +                    for (Row row : partitionUpdate)
 +                        cells += Iterables.size(row.cells());
 +                }
 +            }
 +        }
 +    }
++
+     public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+         try
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+ 
+             for (int i = 0 ; i < 5 ; i++)
+             {
+                 new RowUpdateBuilder(cfs.metadata, 0, "k")
+                     .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                     .build()
+                     .apply();
+ 
+                 if (i == 2)
+                 {
+                     try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                     {
+                         cfs.forceBlockingFlush();
+                     }
+                     catch (Throwable t)
+                     {
+                         // expected. Cause (after some wrappings) should be a write error
+                         while (!(t instanceof FSWriteError))
+                             t = t.getCause();
+                     }
+                 }
+                 else
+                     cfs.forceBlockingFlush();
+             }
+         }
+         finally
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+         }
+ 
+         CommitLog.instance.sync(true);
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+         // If retries work subsequent flushes should clear up error and this should change to expect 0.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+             throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         for (int i = 0 ; i < 5 ; i++)
+         {
+             new RowUpdateBuilder(cfs.metadata, 0, "k")
+                 .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                 .build()
+                 .apply();
+ 
+             Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+             if (i == 2)
+                 current.makeUnflushable();
+ 
+             flushAction.accept(cfs, current);
+         }
+         if (performCompaction)
+             cfs.forceMajorCompaction();
+         // Make sure metadata saves and reads fine
+         for (SSTableReader reader : cfs.getLiveSSTables())
+             reader.reloadSSTableMetadata();
+ 
+         CommitLog.instance.sync(true);
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+         // persisted all data in the commit log. Because we know there was an error, there must be something left to
+         // replay.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+     {
+         try
+         {
+             cfs.forceBlockingFlush();
+         }
+         catch (Throwable t)
+         {
+             // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+             while (!(t instanceof FSWriteError))
+                 t = t.getCause();
+             // Wait for started flushes to complete.
+             cfs.switchMemtableIfCurrent(current);
+         }
+     };
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+     {
+         // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+         // flushed data.
+         // This does not stop on errors and should retain segments for which flushing failed.
+         CommitLog.instance.forceRecycleAllSegments();
+ 
+         // Wait for started flushes to complete.
+         cfs.switchMemtableIfCurrent(current);
+     };
+ 
+     @Test
+     public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, true);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, true);
+     }
  }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index 1668ddc,479e4e2..84e3e05
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@@ -299,14 -298,10 +299,11 @@@ public class TrackerTes
          Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
  
          SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
 -        tracker.replaceFlushed(prev2, Collections.singleton(reader));
 +        tracker.replaceFlushed(prev2, singleton(reader));
          Assert.assertEquals(1, tracker.getView().sstables.size());
-         Assert.assertEquals(1, tracker.getView().premature.size());
-         tracker.permitCompactionOfFlushed(singleton(reader));
-         Assert.assertEquals(0, tracker.getView().premature.size());
 -        Assert.assertEquals(1, listener.received.size());
 -        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
 +        Assert.assertEquals(2, listener.received.size());
 +        Assert.assertEquals(prev2, ((MemtableDiscardedNotification) listener.received.get(0)).memtable);
 +        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(1)).added);
          listener.received.clear();
          Assert.assertTrue(reader.isKeyCacheSetup());
          Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
@@@ -324,13 -319,10 +321,12 @@@
          Assert.assertEquals(0, tracker.getView().sstables.size());
          Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
          Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
-         System.out.println(listener.received);
-         Assert.assertEquals(4, listener.received.size());
 -        Assert.assertEquals(3, listener.received.size());
 -        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
 -        Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
 -        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
++        Assert.assertEquals(5, listener.received.size());
 +        Assert.assertEquals(prev1, ((MemtableSwitchedNotification) listener.received.get(0)).memtable);
 +        Assert.assertEquals(prev1, ((MemtableDiscardedNotification) listener.received.get(1)).memtable);
-         Assert.assertTrue(listener.received.get(2) instanceof SSTableDeletingNotification);
-         Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(3)).removed.size());
++        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(2)).added);
++        Assert.assertTrue(listener.received.get(3) instanceof SSTableDeletingNotification);
++        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(4)).removed.size());
          DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index a3382c4,de12d57..4bd4489
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@@ -32,7 -32,8 +32,8 @@@ import org.apache.cassandra.SchemaLoade
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.db.SerializationHeader;
  import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.dht.RandomPartitioner;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;
@@@ -85,8 -86,7 +86,7 @@@ public class MetadataSerializerTes
  
          CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
          MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                           .commitLogLowerBound(cllb)
-                                           .commitLogUpperBound(club);
 -                                          .commitLogIntervals(new IntervalSet(cllb, club));
++                                          .commitLogIntervals(new IntervalSet<>(cllb, club));
  
          String partitioner = RandomPartitioner.class.getCanonicalName();
          double bfFpChance = 0.1;


[05/23] cassandra git commit: Disable passing control to post-flush after flush failure to prevent data loss.

Posted by sl...@apache.org.
Disable passing control to post-flush after flush failure to prevent
data loss.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828

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

Branch: refs/heads/trunk
Commit: bd6654733dded3513c2c7acf96df2c364b0c043e
Parents: bc0d1da
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 3 11:32:48 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:35:25 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  45 ++++--
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  87 ++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 311 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8ecc787..1275631 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/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 b64d5de..6e82745 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -99,6 +99,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                               new NamedThreadFactory("MemtablePostFlush"),
                                                                                               "internal");
 
+    // If a flush fails with an error the post-flush is never allowed to continue. This stores the error that caused it
+    // to be able to show an error on following flushes instead of blindly continuing.
+    private static volatile FSWriteError previousFlushFailure = null;
+
     private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
                                                                                             StageManager.KEEPALIVE,
                                                                                             TimeUnit.SECONDS,
@@ -869,12 +873,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         synchronized (data)
         {
+            if (previousFlushFailure != null)
+                throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                              + "no flushes can be carried out until the node is restarted.",
+                                                previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            flushExecutor.execute(flush);
+            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            flushExecutor.submit(flushTask);
             ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
             postFlushExecutor.submit(task);
-            return task;
+
+            @SuppressWarnings("unchecked")
+            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            return future;
         }
     }
 
@@ -967,7 +979,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         final ReplayPosition lastReplayPosition;
-        volatile FSWriteError flushFailure = null;
 
         private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
         {
@@ -1010,16 +1021,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             // must check lastReplayPosition != null because Flush may find that all memtables are clean
             // and so not set a lastReplayPosition
-            // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (lastReplayPosition != null && flushFailure == null)
+            if (lastReplayPosition != null)
             {
                 CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
             }
 
             metric.pendingFlushes.dec();
-
-            if (flushFailure != null)
-                throw flushFailure;
         }
     }
 
@@ -1127,16 +1134,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
                     reclaim(memtable);
                 }
+
+                // signal the post-flush we've done our work
+                // Note: This should not be done in case of error. Read more below.
+                postFlush.latch.countDown();
             }
             catch (FSWriteError e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
-                postFlush.flushFailure = e;
+                // The call above may kill the process or the transports, or ignore the error.
+                // In any case we should not be passing on control to post-flush as a subsequent succeeding flush
+                // could mask the error and:
+                //   - let the commit log discard unpersisted data, resulting in data loss
+                //   - let truncations proceed, with the possibility of resurrecting the unflushed data
+                //   - let snapshots succeed with incomplete data
+
+                // Not passing control on means that all flushes from the moment of failure cannot complete
+                // (including snapshots).
+                // If the disk failure policy is ignore, this will cause memtables and the commit log to grow
+                // unboundedly until the node eventually fails.
+                previousFlushFailure = e;
+                throw e;
             }
-
-            // signal the post-flush we've done our work
-            postFlush.latch.countDown();
         }
 
         private void reclaim(final Memtable memtable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
new file mode 100644
index 0000000..c0023dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOError;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories.DataDirectory;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.CommitLogSegment;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSWriteError;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+ */
+public class OutOfSpaceBase extends CQLTester
+{
+    public void makeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+    }
+
+    public void markDirectoriesUnwriteable()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+    }
+
+    public void flushAndExpectError() throws InterruptedException, ExecutionException
+    {
+        try
+        {
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+            fail("FSWriteError expected.");
+        }
+        catch (ExecutionException e)
+        {
+            // Correct path.
+            Assert.assertTrue(e.getCause() instanceof FSWriteError);
+        }
+
+        // Make sure commit log wasn't discarded.
+        UUID cfid = currentTableMetadata().cfId;
+        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+            if (segment.getDirtyCFIDs().contains(cfid))
+                return;
+        fail("Expected commit log to remain dirty for the affected table.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
new file mode 100644
index 0000000..46d71e4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceDieTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableDie() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
+            flushAndExpectError();
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
new file mode 100644
index 0000000..854de80
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceIgnoreTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableIgnore() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+            flushAndExpectError();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
new file mode 100644
index 0000000..b48df56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceStopTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableStop() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
+            flushAndExpectError();
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
deleted file mode 100644
index 8304aff..0000000
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ /dev/null
@@ -1,157 +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.cql3;
-
-import static junit.framework.Assert.fail;
-
-import java.io.IOError;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config.DiskFailurePolicy;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class OutOfSpaceTest extends CQLTester
-{
-    @Test
-    public void testFlushUnwriteableDie() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
-            flushAndExpectError();
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableStop() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
-            flushAndExpectError();
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableIgnore() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
-            flushAndExpectError();
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-
-        // Next flush should succeed.
-        makeTable();
-        flush();
-    }
-
-    public void makeTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-    }
-
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.directories.getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
-    public void flushAndExpectError() throws InterruptedException, ExecutionException
-    {
-        try
-        {
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
-            fail("FSWriteError expected.");
-        }
-        catch (ExecutionException e)
-        {
-            // Correct path.
-            Assert.assertTrue(e.getCause() instanceof FSWriteError);
-        }
-
-        // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
-        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
-                return;
-        fail("Expected commit log to remain dirty for the affected table.");
-    }
-}


[04/23] cassandra git commit: Disable passing control to post-flush after flush failure to prevent data loss.

Posted by sl...@apache.org.
Disable passing control to post-flush after flush failure to prevent
data loss.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828

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

Branch: refs/heads/cassandra-3.9
Commit: bd6654733dded3513c2c7acf96df2c364b0c043e
Parents: bc0d1da
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 3 11:32:48 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:35:25 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  45 ++++--
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  87 ++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 311 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8ecc787..1275631 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/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 b64d5de..6e82745 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -99,6 +99,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                               new NamedThreadFactory("MemtablePostFlush"),
                                                                                               "internal");
 
+    // If a flush fails with an error the post-flush is never allowed to continue. This stores the error that caused it
+    // to be able to show an error on following flushes instead of blindly continuing.
+    private static volatile FSWriteError previousFlushFailure = null;
+
     private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
                                                                                             StageManager.KEEPALIVE,
                                                                                             TimeUnit.SECONDS,
@@ -869,12 +873,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         synchronized (data)
         {
+            if (previousFlushFailure != null)
+                throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                              + "no flushes can be carried out until the node is restarted.",
+                                                previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            flushExecutor.execute(flush);
+            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            flushExecutor.submit(flushTask);
             ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
             postFlushExecutor.submit(task);
-            return task;
+
+            @SuppressWarnings("unchecked")
+            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            return future;
         }
     }
 
@@ -967,7 +979,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         final ReplayPosition lastReplayPosition;
-        volatile FSWriteError flushFailure = null;
 
         private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
         {
@@ -1010,16 +1021,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             // must check lastReplayPosition != null because Flush may find that all memtables are clean
             // and so not set a lastReplayPosition
-            // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (lastReplayPosition != null && flushFailure == null)
+            if (lastReplayPosition != null)
             {
                 CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
             }
 
             metric.pendingFlushes.dec();
-
-            if (flushFailure != null)
-                throw flushFailure;
         }
     }
 
@@ -1127,16 +1134,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
                     reclaim(memtable);
                 }
+
+                // signal the post-flush we've done our work
+                // Note: This should not be done in case of error. Read more below.
+                postFlush.latch.countDown();
             }
             catch (FSWriteError e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
-                postFlush.flushFailure = e;
+                // The call above may kill the process or the transports, or ignore the error.
+                // In any case we should not be passing on control to post-flush as a subsequent succeeding flush
+                // could mask the error and:
+                //   - let the commit log discard unpersisted data, resulting in data loss
+                //   - let truncations proceed, with the possibility of resurrecting the unflushed data
+                //   - let snapshots succeed with incomplete data
+
+                // Not passing control on means that all flushes from the moment of failure cannot complete
+                // (including snapshots).
+                // If the disk failure policy is ignore, this will cause memtables and the commit log to grow
+                // unboundedly until the node eventually fails.
+                previousFlushFailure = e;
+                throw e;
             }
-
-            // signal the post-flush we've done our work
-            postFlush.latch.countDown();
         }
 
         private void reclaim(final Memtable memtable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
new file mode 100644
index 0000000..c0023dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOError;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories.DataDirectory;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.CommitLogSegment;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSWriteError;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+ */
+public class OutOfSpaceBase extends CQLTester
+{
+    public void makeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+    }
+
+    public void markDirectoriesUnwriteable()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+    }
+
+    public void flushAndExpectError() throws InterruptedException, ExecutionException
+    {
+        try
+        {
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+            fail("FSWriteError expected.");
+        }
+        catch (ExecutionException e)
+        {
+            // Correct path.
+            Assert.assertTrue(e.getCause() instanceof FSWriteError);
+        }
+
+        // Make sure commit log wasn't discarded.
+        UUID cfid = currentTableMetadata().cfId;
+        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+            if (segment.getDirtyCFIDs().contains(cfid))
+                return;
+        fail("Expected commit log to remain dirty for the affected table.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
new file mode 100644
index 0000000..46d71e4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceDieTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableDie() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
+            flushAndExpectError();
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
new file mode 100644
index 0000000..854de80
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceIgnoreTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableIgnore() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+            flushAndExpectError();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
new file mode 100644
index 0000000..b48df56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceStopTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableStop() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
+            flushAndExpectError();
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
deleted file mode 100644
index 8304aff..0000000
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ /dev/null
@@ -1,157 +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.cql3;
-
-import static junit.framework.Assert.fail;
-
-import java.io.IOError;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config.DiskFailurePolicy;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class OutOfSpaceTest extends CQLTester
-{
-    @Test
-    public void testFlushUnwriteableDie() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
-            flushAndExpectError();
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableStop() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
-            flushAndExpectError();
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableIgnore() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
-            flushAndExpectError();
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-
-        // Next flush should succeed.
-        makeTable();
-        flush();
-    }
-
-    public void makeTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-    }
-
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.directories.getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
-    public void flushAndExpectError() throws InterruptedException, ExecutionException
-    {
-        try
-        {
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
-            fail("FSWriteError expected.");
-        }
-        catch (ExecutionException e)
-        {
-            // Correct path.
-            Assert.assertTrue(e.getCause() instanceof FSWriteError);
-        }
-
-        // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
-        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
-                return;
-        fail("Expected commit log to remain dirty for the affected table.");
-    }
-}


[21/23] cassandra git commit: Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/Tracker.java
index b1c706e,5a3d524..f464e08
--- a/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/Tracker.java
@@@ -353,35 -347,13 +349,16 @@@ public class Tracke
  
          Throwable fail;
          fail = updateSizeTracking(emptySet(), sstables, null);
 +
 +        notifyDiscarded(memtable);
 +
-         maybeFail(fail);
-     }
- 
-     /**
-      * permit compaction of the provided sstable; this translates to notifying compaction
-      * strategies of its existence, and potentially submitting a background task
-      */
-     public void permitCompactionOfFlushed(Collection<SSTableReader> sstables)
-     {
-         if (sstables.isEmpty())
-             return;
+         // TODO: if we're invalidated, should we notifyadded AND removed, or just skip both?
+         fail = notifyAdded(sstables, fail);
  
-         apply(View.permitCompactionOfFlushed(sstables));
- 
-         if (isDummy())
-             return;
- 
-         if (cfstore.isValid())
-         {
-             notifyAdded(sstables);
-             CompactionManager.instance.submitBackground(cfstore);
-         }
-         else
-         {
+         if (!isDummy() && !cfstore.isValid())
              dropSSTables();
-         }
+ 
+         maybeFail(fail);
      }
  
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/lifecycle/View.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/lifecycle/View.java
index a5c781d,4b3aae0..b26426d
--- a/src/java/org/apache/cassandra/db/lifecycle/View.java
+++ b/src/java/org/apache/cassandra/db/lifecycle/View.java
@@@ -40,7 -39,7 +39,6 @@@ import static com.google.common.collect
  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 com.google.common.collect.Iterables.transform;
  import static org.apache.cassandra.db.lifecycle.Helpers.emptySet;
  import static org.apache.cassandra.db.lifecycle.Helpers.filterOut;
  import static org.apache.cassandra.db.lifecycle.Helpers.replace;
@@@ -336,14 -333,12 +332,12 @@@ public class Vie
                  List<Memtable> flushingMemtables = copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
                  assert flushingMemtables.size() == view.flushingMemtables.size() - 1;
  
 -                if (flushed == null || flushed.isEmpty())
 +                if (flushed == null || Iterables.isEmpty(flushed))
                      return new View(view.liveMemtables, flushingMemtables, view.sstablesMap,
-                                     view.compactingMap, view.premature, view.intervalTree);
+                                     view.compactingMap, view.intervalTree);
  
                  Map<SSTableReader, SSTableReader> sstableMap = replace(view.sstablesMap, emptySet(), flushed);
-                 Map<SSTableReader, SSTableReader> compactingMap = replace(view.compactingMap, emptySet(), flushed);
-                 Set<SSTableReader> premature = replace(view.premature, emptySet(), flushed);
-                 return new View(view.liveMemtables, flushingMemtables, sstableMap, compactingMap, premature,
+                 return new View(view.liveMemtables, flushingMemtables, sstableMap, view.compactingMap,
                                  SSTableIntervalTree.build(sstableMap.keySet()));
              }
          };

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/format/big/BigFormat.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 505de49,a683513..14e391b
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@@ -24,7 -24,8 +24,8 @@@ import java.util.*
  import com.google.common.collect.Maps;
  
  import org.apache.cassandra.db.TypeSizes;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.format.Version;
@@@ -35,6 -36,8 +36,8 @@@ import org.apache.cassandra.utils.ByteB
  import org.apache.cassandra.utils.EstimatedHistogram;
  import org.apache.cassandra.utils.StreamingHistogram;
  
 -import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.replayPositionSetSerializer;
++import static org.apache.cassandra.io.sstable.metadata.StatsMetadata.commitLogPositionSetSerializer;
+ 
  /**
   * Serializer for SSTable from legacy versions
   */
@@@ -55,7 -58,7 +58,7 @@@ public class LegacyMetadataSerializer e
  
          EstimatedHistogram.serializer.serialize(stats.estimatedPartitionSize, out);
          EstimatedHistogram.serializer.serialize(stats.estimatedColumnCount, out);
-         CommitLogPosition.serializer.serialize(stats.commitLogUpperBound, out);
 -        ReplayPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
++        CommitLogPosition.serializer.serialize(stats.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE), out);
          out.writeLong(stats.minTimestamp);
          out.writeLong(stats.maxTimestamp);
          out.writeInt(stats.maxLocalDeletionTime);
@@@ -72,7 -75,9 +75,9 @@@
          for (ByteBuffer value : stats.maxClusteringValues)
              ByteBufferUtil.writeWithShortLength(value, out);
          if (version.hasCommitLogLowerBound())
-             CommitLogPosition.serializer.serialize(stats.commitLogLowerBound, out);
 -            ReplayPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
++            CommitLogPosition.serializer.serialize(stats.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE), out);
+         if (version.hasCommitLogIntervals())
 -            replayPositionSetSerializer.serialize(stats.commitLogIntervals, out);
++            commitLogPositionSetSerializer.serialize(stats.commitLogIntervals, out);
      }
  
      /**
@@@ -120,7 -125,12 +125,12 @@@
                      maxColumnNames.add(ByteBufferUtil.readWithShortLength(in));
  
                  if (descriptor.version.hasCommitLogLowerBound())
 -                    commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
 -                IntervalSet<ReplayPosition> commitLogIntervals;
 +                    commitLogLowerBound = CommitLogPosition.serializer.deserialize(in);
++                IntervalSet<CommitLogPosition> commitLogIntervals;
+                 if (descriptor.version.hasCommitLogIntervals())
 -                    commitLogIntervals = replayPositionSetSerializer.deserialize(in);
++                    commitLogIntervals = commitLogPositionSetSerializer.deserialize(in);
+                 else
+                     commitLogIntervals = new IntervalSet<>(commitLogLowerBound, commitLogUpperBound);
  
                  if (types.contains(MetadataType.VALIDATION))
                      components.put(MetadataType.VALIDATION,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index 299bc87,1ff2ca8..196cfbf
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@@ -20,16 -20,18 +20,16 @@@ package org.apache.cassandra.io.sstable
  import java.nio.ByteBuffer;
  import java.util.ArrayList;
  import java.util.Collections;
 -import java.util.HashSet;
  import java.util.List;
  import java.util.Map;
 -import java.util.Set;
  
  import com.google.common.collect.Maps;
- import com.google.common.collect.Ordering;
  
  import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
  import com.clearspring.analytics.stream.cardinality.ICardinality;
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
  import org.apache.cassandra.db.marshal.AbstractType;
  import org.apache.cassandra.db.partitions.PartitionStatisticsCollector;
  import org.apache.cassandra.db.rows.Cell;
@@@ -88,8 -89,7 +87,7 @@@ public class MetadataCollector implemen
      protected EstimatedHistogram estimatedPartitionSize = defaultPartitionSizeHistogram();
      // TODO: cound the number of row per partition (either with the number of cells, or instead)
      protected EstimatedHistogram estimatedCellPerPartitionCount = defaultCellPerPartitionCountHistogram();
-     protected CommitLogPosition commitLogLowerBound = CommitLogPosition.NONE;
-     protected CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE;
 -    protected IntervalSet commitLogIntervals = IntervalSet.empty();
++    protected IntervalSet<CommitLogPosition> commitLogIntervals = IntervalSet.empty();
      protected final MinMaxLongTracker timestampTracker = new MinMaxLongTracker();
      protected final MinMaxIntTracker localDeletionTimeTracker = new MinMaxIntTracker(Cell.NO_DELETION_TIME, Cell.NO_DELETION_TIME);
      protected final MinMaxIntTracker ttlTracker = new MinMaxIntTracker(Cell.NO_TTL, Cell.NO_TTL);
@@@ -123,23 -123,13 +121,13 @@@
      {
          this(comparator);
  
-         CommitLogPosition min = null, max = null;
 -        IntervalSet.Builder intervals = new IntervalSet.Builder();
++        IntervalSet.Builder<CommitLogPosition> intervals = new IntervalSet.Builder<>();
          for (SSTableReader sstable : sstables)
          {
-             if (min == null)
-             {
-                 min = sstable.getSSTableMetadata().commitLogLowerBound;
-                 max = sstable.getSSTableMetadata().commitLogUpperBound;
-             }
-             else
-             {
-                 min = Ordering.natural().min(min, sstable.getSSTableMetadata().commitLogLowerBound);
-                 max = Ordering.natural().max(max, sstable.getSSTableMetadata().commitLogUpperBound);
-             }
+             intervals.addAll(sstable.getSSTableMetadata().commitLogIntervals);
          }
  
-         commitLogLowerBound(min);
-         commitLogUpperBound(max);
+         commitLogIntervals(intervals.build());
          sstableLevel(level);
      }
  
@@@ -226,15 -216,9 +214,9 @@@
          ttlTracker.update(newTTL);
      }
  
-     public MetadataCollector commitLogLowerBound(CommitLogPosition commitLogLowerBound)
 -    public MetadataCollector commitLogIntervals(IntervalSet commitLogIntervals)
++    public MetadataCollector commitLogIntervals(IntervalSet<CommitLogPosition> commitLogIntervals)
      {
-         this.commitLogLowerBound = commitLogLowerBound;
-         return this;
-     }
- 
-     public MetadataCollector commitLogUpperBound(CommitLogPosition commitLogUpperBound)
-     {
-         this.commitLogUpperBound = commitLogUpperBound;
+         this.commitLogIntervals = commitLogIntervals;
          return this;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
index e765235,9971eaa..c83c2cf
--- a/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/StatsMetadata.java
@@@ -26,7 -27,8 +27,8 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.commons.lang3.builder.EqualsBuilder;
  import org.apache.commons.lang3.builder.HashCodeBuilder;
  import org.apache.cassandra.db.TypeSizes;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.io.util.DataInputPlus;
  import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.utils.ByteBufferUtil;
@@@ -39,11 -41,11 +41,11 @@@ import org.apache.cassandra.utils.Strea
  public class StatsMetadata extends MetadataComponent
  {
      public static final IMetadataComponentSerializer serializer = new StatsMetadataSerializer();
 -    public static final ISerializer<IntervalSet<ReplayPosition>> replayPositionSetSerializer = IntervalSet.serializer(ReplayPosition.serializer);
++    public static final ISerializer<IntervalSet<CommitLogPosition>> commitLogPositionSetSerializer = IntervalSet.serializer(CommitLogPosition.serializer);
  
      public final EstimatedHistogram estimatedPartitionSize;
      public final EstimatedHistogram estimatedColumnCount;
-     public final CommitLogPosition commitLogLowerBound;
-     public final CommitLogPosition commitLogUpperBound;
 -    public final IntervalSet<ReplayPosition> commitLogIntervals;
++    public final IntervalSet<CommitLogPosition> commitLogIntervals;
      public final long minTimestamp;
      public final long maxTimestamp;
      public final int minLocalDeletionTime;
@@@ -62,8 -64,7 +64,7 @@@
  
      public StatsMetadata(EstimatedHistogram estimatedPartitionSize,
                           EstimatedHistogram estimatedColumnCount,
-                          CommitLogPosition commitLogLowerBound,
-                          CommitLogPosition commitLogUpperBound,
 -                         IntervalSet<ReplayPosition> commitLogIntervals,
++                         IntervalSet<CommitLogPosition> commitLogIntervals,
                           long minTimestamp,
                           long maxTimestamp,
                           int minLocalDeletionTime,
@@@ -239,7 -235,7 +235,7 @@@
              int size = 0;
              size += EstimatedHistogram.serializer.serializedSize(component.estimatedPartitionSize);
              size += EstimatedHistogram.serializer.serializedSize(component.estimatedColumnCount);
-             size += CommitLogPosition.serializer.serializedSize(component.commitLogUpperBound);
 -            size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE));
++            size += CommitLogPosition.serializer.serializedSize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE));
              if (version.storeRows())
                  size += 8 + 8 + 4 + 4 + 4 + 4 + 8 + 8; // mix/max timestamp(long), min/maxLocalDeletionTime(int), min/max TTL, compressionRatio(double), repairedAt (long)
              else
@@@ -258,7 -254,9 +254,9 @@@
              if (version.storeRows())
                  size += 8 + 8; // totalColumnsSet, totalRows
              if (version.hasCommitLogLowerBound())
-                 size += CommitLogPosition.serializer.serializedSize(component.commitLogLowerBound);
 -                size += ReplayPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE));
++                size += CommitLogPosition.serializer.serializedSize(component.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE));
+             if (version.hasCommitLogIntervals())
 -                size += replayPositionSetSerializer.serializedSize(component.commitLogIntervals);
++                size += commitLogPositionSetSerializer.serializedSize(component.commitLogIntervals);
              return size;
          }
  
@@@ -266,7 -264,7 +264,7 @@@
          {
              EstimatedHistogram.serializer.serialize(component.estimatedPartitionSize, out);
              EstimatedHistogram.serializer.serialize(component.estimatedColumnCount, out);
-             CommitLogPosition.serializer.serialize(component.commitLogUpperBound, out);
 -            ReplayPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(ReplayPosition.NONE), out);
++            CommitLogPosition.serializer.serialize(component.commitLogIntervals.upperBound().orElse(CommitLogPosition.NONE), out);
              out.writeLong(component.minTimestamp);
              out.writeLong(component.maxTimestamp);
              if (version.storeRows())
@@@ -296,7 -294,9 +294,9 @@@
              }
  
              if (version.hasCommitLogLowerBound())
-                 CommitLogPosition.serializer.serialize(component.commitLogLowerBound, out);
 -                ReplayPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(ReplayPosition.NONE), out);
++                CommitLogPosition.serializer.serialize(component.commitLogIntervals.lowerBound().orElse(CommitLogPosition.NONE), out);
+             if (version.hasCommitLogIntervals())
 -                replayPositionSetSerializer.serialize(component.commitLogIntervals, out);
++                commitLogPositionSetSerializer.serialize(component.commitLogIntervals, out);
          }
  
          public StatsMetadata deserialize(Version version, DataInputPlus in) throws IOException
@@@ -337,7 -337,12 +337,12 @@@
              long totalRows = version.storeRows() ? in.readLong() : -1L;
  
              if (version.hasCommitLogLowerBound())
 -                commitLogLowerBound = ReplayPosition.serializer.deserialize(in);
 -            IntervalSet<ReplayPosition> commitLogIntervals;
 +                commitLogLowerBound = CommitLogPosition.serializer.deserialize(in);
++            IntervalSet<CommitLogPosition> commitLogIntervals;
+             if (version.hasCommitLogIntervals())
 -                commitLogIntervals = replayPositionSetSerializer.deserialize(in);
++                commitLogIntervals = commitLogPositionSetSerializer.deserialize(in);
+             else
 -                commitLogIntervals = new IntervalSet<ReplayPosition>(commitLogLowerBound, commitLogUpperBound);
++                commitLogIntervals = new IntervalSet<CommitLogPosition>(commitLogLowerBound, commitLogUpperBound);
  
              return new StatsMetadata(partitionSizes,
                                       columnCounts,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 3c8ba64,5f7513f..6686684
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -112,15 -70,11 +112,14 @@@ public class SSTableMetadataViewe
                      out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
-                     out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
-                     out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
+                     out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
 +                    out.printf("totalColumnsSet: %s%n", stats.totalColumnsSet);
 +                    out.printf("totalRows: %s%n", stats.totalRows);
                      out.println("Estimated tombstone drop times:");
 -                    for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
 +
 +                    for (Map.Entry<Number, long[]> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                      {
 -                        out.printf("%-10s:%10s%n",entry.getKey().intValue(), entry.getValue());
 +                        out.printf("%-10s:%10s%n",entry.getKey().intValue(), entry.getValue()[0]);
                      }
                      printHistograms(stats, out);
                  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --cc test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index 239077e,02b26c7..2858597
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@@ -250,9 -245,9 +250,10 @@@ public class CommitLogStressTes
              }
              verifySizes(commitLog);
  
-             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId, discardedPos);
+             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
 -                    ReplayPosition.NONE, discardedPos);
++                    CommitLogPosition.NONE, discardedPos);
              threads.clear();
 +
              System.out.format("Discarded at %s\n", discardedPos);
              verifySizes(commitLog);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/Util.java
index cd709d5,d04ca9b..7bcee7a
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@@ -677,9 -648,26 +679,26 @@@ public class Uti
          }
  
          private UnfilteredPartitionIterator queryStorageInternal(ColumnFamilyStore cfs,
 -                                                                 ReadOrderGroup orderGroup)
 +                                                                 ReadExecutionController controller)
          {
 -            return queryStorage(cfs, orderGroup);
 +            return queryStorage(cfs, controller);
          }
      }
+ 
+     public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)
+     {
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+         return () -> BlacklistedDirectories.clearUnwritableUnsafe();
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 23ec58b,9a0ddb8..6ab7d46
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@@ -20,9 -20,13 +20,10 @@@ package org.apache.cassandra.db.commitl
  
  import java.io.*;
  import java.nio.ByteBuffer;
 -import java.util.Arrays;
 -import java.util.Collection;
 -import java.util.Collections;
 -import java.util.UUID;
 +import java.util.*;
  import java.util.concurrent.Callable;
  import java.util.concurrent.ExecutionException;
+ import java.util.function.BiConsumer;
  import java.util.zip.CRC32;
  import java.util.zip.Checksum;
  
@@@ -35,34 -40,27 +36,34 @@@ import org.junit.runners.Parameterized.
  
  import org.apache.cassandra.SchemaLoader;
  import org.apache.cassandra.Util;
 +import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.config.ParameterizedClass;
- import org.apache.cassandra.db.ColumnFamilyStore;
- import org.apache.cassandra.db.Keyspace;
- import org.apache.cassandra.db.Mutation;
- import org.apache.cassandra.db.RowUpdateBuilder;
+ import org.apache.cassandra.config.Config.DiskFailurePolicy;
+ import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
  import org.apache.cassandra.db.compaction.CompactionManager;
  import org.apache.cassandra.db.marshal.AsciiType;
  import org.apache.cassandra.db.marshal.BytesType;
 +import org.apache.cassandra.db.partitions.PartitionUpdate;
 +import org.apache.cassandra.db.rows.Row;
  import org.apache.cassandra.exceptions.ConfigurationException;
+ import org.apache.cassandra.io.FSWriteError;
  import org.apache.cassandra.io.compress.DeflateCompressor;
  import org.apache.cassandra.io.compress.LZ4Compressor;
  import org.apache.cassandra.io.compress.SnappyCompressor;
+ import org.apache.cassandra.io.sstable.format.SSTableReader;
  import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.schema.KeyspaceParams;
 -import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.security.EncryptionContext;
 +import org.apache.cassandra.security.EncryptionContextGenerator;
 +import org.apache.cassandra.utils.Hex;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.KillerForTests;
 +import org.apache.cassandra.utils.Pair;
  import org.apache.cassandra.utils.vint.VIntCoding;
  
 +import org.junit.After;
  import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertTrue;
@@@ -247,13 -228,13 +248,13 @@@ public class CommitLogTes
                        .build();
          CommitLog.instance.add(m2);
  
 -        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 +        assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
  
          UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-         CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
 -        // Assert we still have both our segment
 -        assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 +        // Assert we still have both our segments
 +        assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
      }
  
      @Test
@@@ -278,9 -258,9 +279,9 @@@
          // "Flush": this won't delete anything
          UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
          CommitLog.instance.sync(true);
-         CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid1, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 +        assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
  
          // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
          Mutation rm2 = new RowUpdateBuilder(cfs2.metadata, 0, "k")
@@@ -298,10 -279,10 +299,10 @@@
          // didn't write anything on cf1 since last flush (and we flush cf2)
  
          UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-         CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getCurrentPosition());
 -        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
++        CommitLog.instance.discardCompletedSegments(cfid2, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
          // Assert we still have both our segment
 -        assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 +        assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
      }
  
      private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String cfName, String colName)
@@@ -545,13 -486,13 +546,13 @@@
              for (int i = 0 ; i < 5 ; i++)
                  CommitLog.instance.add(m2);
  
 -            assertEquals(2, CommitLog.instance.activeSegments());
 -            ReplayPosition position = CommitLog.instance.getContext();
 -            for (Keyspace ks : Keyspace.system())
 -                for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
 -                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, ReplayPosition.NONE, position);
 -            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, ReplayPosition.NONE, position);
 -            assertEquals(1, CommitLog.instance.activeSegments());
 +            assertEquals(2, CommitLog.instance.segmentManager.getActiveSegments().size());
 +            CommitLogPosition position = CommitLog.instance.getCurrentPosition();
 +            for (Keyspace keyspace : Keyspace.system())
 +                for (ColumnFamilyStore syscfs : keyspace.getColumnFamilyStores())
-                     CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-             CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
++                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, CommitLogPosition.NONE, position);
++            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, CommitLogPosition.NONE, position);
 +            assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
          }
          finally
          {
@@@ -589,108 -530,136 +590,240 @@@
      }
  
      @Test
 +    public void replaySimple() throws IOException
 +    {
 +        int cellCount = 0;
 +        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 +        final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k1")
 +                             .clustering("bytes")
 +                             .add("val", bytes("this is a string"))
 +                             .build();
 +        cellCount += 1;
 +        CommitLog.instance.add(rm1);
 +
 +        final Mutation rm2 = new RowUpdateBuilder(cfs.metadata, 0, "k2")
 +                             .clustering("bytes")
 +                             .add("val", bytes("this is a string"))
 +                             .build();
 +        cellCount += 1;
 +        CommitLog.instance.add(rm2);
 +
 +        CommitLog.instance.sync(true);
 +
 +        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, CommitLogPosition.NONE, cfs.metadata);
 +        List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
 +        Assert.assertFalse(activeSegments.isEmpty());
 +
 +        File[] files = new File(CommitLog.instance.segmentManager.storageDirectory).listFiles((file, name) -> activeSegments.contains(name));
 +        replayer.replayFiles(files);
 +
 +        assertEquals(cellCount, replayer.cells);
 +    }
 +
 +    @Test
 +    public void replayWithDiscard() throws IOException
 +    {
 +        int cellCount = 0;
 +        int max = 1024;
 +        int discardPosition = (int)(max * .8); // an arbitrary number of entries that we'll skip on the replay
 +        CommitLogPosition commitLogPosition = null;
 +        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
 +
 +        for (int i = 0; i < max; i++)
 +        {
 +            final Mutation rm1 = new RowUpdateBuilder(cfs.metadata, 0, "k" + 1)
 +                                 .clustering("bytes")
 +                                 .add("val", bytes("this is a string"))
 +                                 .build();
 +            CommitLogPosition position = CommitLog.instance.add(rm1);
 +
 +            if (i == discardPosition)
 +                commitLogPosition = position;
 +            if (i > discardPosition)
 +            {
 +                cellCount += 1;
 +            }
 +        }
 +
 +        CommitLog.instance.sync(true);
 +
 +        SimpleCountingReplayer replayer = new SimpleCountingReplayer(CommitLog.instance, commitLogPosition, cfs.metadata);
 +        List<String> activeSegments = CommitLog.instance.getActiveSegmentNames();
 +        Assert.assertFalse(activeSegments.isEmpty());
 +
 +        File[] files = new File(CommitLog.instance.segmentManager.storageDirectory).listFiles((file, name) -> activeSegments.contains(name));
 +        replayer.replayFiles(files);
 +
 +        assertEquals(cellCount, replayer.cells);
 +    }
 +
 +    class SimpleCountingReplayer extends CommitLogReplayer
 +    {
 +        private final CommitLogPosition filterPosition;
 +        private final CFMetaData metadata;
 +        int cells;
 +        int skipped;
 +
 +        SimpleCountingReplayer(CommitLog commitLog, CommitLogPosition filterPosition, CFMetaData cfm)
 +        {
 +            super(commitLog, filterPosition, Collections.emptyMap(), ReplayFilter.create());
 +            this.filterPosition = filterPosition;
 +            this.metadata = cfm;
 +        }
 +
 +        @SuppressWarnings("resource")
 +        @Override
 +        public void handleMutation(Mutation m, int size, int entryLocation, CommitLogDescriptor desc)
 +        {
 +            if (entryLocation <= filterPosition.position)
 +            {
 +                // Skip over this mutation.
 +                skipped++;
 +                return;
 +            }
 +            for (PartitionUpdate partitionUpdate : m.getPartitionUpdates())
 +            {
 +                // Only process mutations for the CF's we're testing against, since we can't deterministically predict
 +                // whether or not system keyspaces will be mutated during a test.
 +                if (partitionUpdate.metadata().cfName.equals(metadata.cfName))
 +                {
 +                    for (Row row : partitionUpdate)
 +                        cells += Iterables.size(row.cells());
 +                }
 +            }
 +        }
 +    }
++
+     public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+         try
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+ 
+             for (int i = 0 ; i < 5 ; i++)
+             {
+                 new RowUpdateBuilder(cfs.metadata, 0, "k")
+                     .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                     .build()
+                     .apply();
+ 
+                 if (i == 2)
+                 {
+                     try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                     {
+                         cfs.forceBlockingFlush();
+                     }
+                     catch (Throwable t)
+                     {
+                         // expected. Cause (after some wrappings) should be a write error
+                         while (!(t instanceof FSWriteError))
+                             t = t.getCause();
+                     }
+                 }
+                 else
+                     cfs.forceBlockingFlush();
+             }
+         }
+         finally
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+         }
+ 
+         CommitLog.instance.sync(true);
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+         // If retries work subsequent flushes should clear up error and this should change to expect 0.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+             throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         for (int i = 0 ; i < 5 ; i++)
+         {
+             new RowUpdateBuilder(cfs.metadata, 0, "k")
+                 .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                 .build()
+                 .apply();
+ 
+             Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+             if (i == 2)
+                 current.makeUnflushable();
+ 
+             flushAction.accept(cfs, current);
+         }
+         if (performCompaction)
+             cfs.forceMajorCompaction();
+         // Make sure metadata saves and reads fine
+         for (SSTableReader reader : cfs.getLiveSSTables())
+             reader.reloadSSTableMetadata();
+ 
+         CommitLog.instance.sync(true);
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+         // persisted all data in the commit log. Because we know there was an error, there must be something left to
+         // replay.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+     {
+         try
+         {
+             cfs.forceBlockingFlush();
+         }
+         catch (Throwable t)
+         {
+             // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+             while (!(t instanceof FSWriteError))
+                 t = t.getCause();
+             // Wait for started flushes to complete.
+             cfs.switchMemtableIfCurrent(current);
+         }
+     };
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+     {
+         // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+         // flushed data.
+         // This does not stop on errors and should retain segments for which flushing failed.
+         CommitLog.instance.forceRecycleAllSegments();
+ 
+         // Wait for started flushes to complete.
+         cfs.switchMemtableIfCurrent(current);
+     };
+ 
+     @Test
+     public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, true);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, true);
+     }
  }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
index 1668ddc,479e4e2..84e3e05
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@@ -299,14 -298,10 +299,11 @@@ public class TrackerTes
          Assert.assertTrue(tracker.getView().flushingMemtables.contains(prev2));
  
          SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
 -        tracker.replaceFlushed(prev2, Collections.singleton(reader));
 +        tracker.replaceFlushed(prev2, singleton(reader));
          Assert.assertEquals(1, tracker.getView().sstables.size());
-         Assert.assertEquals(1, tracker.getView().premature.size());
-         tracker.permitCompactionOfFlushed(singleton(reader));
-         Assert.assertEquals(0, tracker.getView().premature.size());
 -        Assert.assertEquals(1, listener.received.size());
 -        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
 +        Assert.assertEquals(2, listener.received.size());
 +        Assert.assertEquals(prev2, ((MemtableDiscardedNotification) listener.received.get(0)).memtable);
 +        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(1)).added);
          listener.received.clear();
          Assert.assertTrue(reader.isKeyCacheSetup());
          Assert.assertEquals(10, cfs.metric.liveDiskSpaceUsed.getCount());
@@@ -324,13 -319,10 +321,12 @@@
          Assert.assertEquals(0, tracker.getView().sstables.size());
          Assert.assertEquals(0, tracker.getView().flushingMemtables.size());
          Assert.assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
-         System.out.println(listener.received);
-         Assert.assertEquals(4, listener.received.size());
 -        Assert.assertEquals(3, listener.received.size());
 -        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
 -        Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
 -        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
++        Assert.assertEquals(5, listener.received.size());
 +        Assert.assertEquals(prev1, ((MemtableSwitchedNotification) listener.received.get(0)).memtable);
 +        Assert.assertEquals(prev1, ((MemtableDiscardedNotification) listener.received.get(1)).memtable);
-         Assert.assertTrue(listener.received.get(2) instanceof SSTableDeletingNotification);
-         Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(3)).removed.size());
++        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(2)).added);
++        Assert.assertTrue(listener.received.get(3) instanceof SSTableDeletingNotification);
++        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(4)).removed.size());
          DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index a3382c4,de12d57..4bd4489
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@@ -32,7 -32,8 +32,8 @@@ import org.apache.cassandra.SchemaLoade
  import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.db.SerializationHeader;
  import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
  import org.apache.cassandra.dht.RandomPartitioner;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;
@@@ -85,8 -86,7 +86,7 @@@ public class MetadataSerializerTes
  
          CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
          MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                           .commitLogLowerBound(cllb)
-                                           .commitLogUpperBound(club);
 -                                          .commitLogIntervals(new IntervalSet(cllb, club));
++                                          .commitLogIntervals(new IntervalSet<>(cllb, club));
  
          String partitioner = RandomPartitioner.class.getCanonicalName();
          double bfFpChance = 0.1;


[17/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/utils/IntegerInterval.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IntegerInterval.java b/src/java/org/apache/cassandra/utils/IntegerInterval.java
new file mode 100644
index 0000000..03ad6e0
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/IntegerInterval.java
@@ -0,0 +1,227 @@
+package org.apache.cassandra.utils;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+/**
+ * Mutable integer interval class, thread-safe.
+ * Represents the interval [lower,upper].
+ */
+public class IntegerInterval
+{
+    volatile long interval;
+    private static AtomicLongFieldUpdater<IntegerInterval> intervalUpdater =
+            AtomicLongFieldUpdater.newUpdater(IntegerInterval.class, "interval");
+
+    private IntegerInterval(long interval)
+    {
+        this.interval = interval;
+    }
+
+    public IntegerInterval(int lower, int upper)
+    {
+        this(make(lower, upper));
+    }
+
+    public IntegerInterval(IntegerInterval src)
+    {
+        this(src.interval);
+    }
+
+    public int lower()
+    {
+        return lower(interval);
+    }
+
+    public int upper()
+    {
+        return upper(interval);
+    }
+
+    /**
+     * Expands the interval to cover the given value by extending one of its sides if necessary.
+     * Mutates this. Thread-safe.
+     */
+    public void expandToCover(int value)
+    {
+        long prev;
+        int lower;
+        int upper;
+        do
+        {
+            prev = interval;
+            upper = upper(prev);
+            lower = lower(prev);
+            if (value > upper) // common case
+                upper = value;
+            else if (value < lower)
+                lower = value;
+        }
+        while (!intervalUpdater.compareAndSet(this, prev, make(lower, upper)));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Long.hashCode(interval);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (getClass() != obj.getClass())
+            return false;
+        IntegerInterval other = (IntegerInterval) obj;
+        return interval == other.interval;
+    }
+
+    public String toString()
+    {
+        long interval = this.interval;
+        return "[" + lower(interval) + "," + upper(interval) + "]";
+    }
+
+    private static long make(int lower, int upper)
+    {
+        assert lower <= upper;
+        return ((lower & 0xFFFFFFFFL) << 32) | upper & 0xFFFFFFFFL;
+    }
+
+    private static int lower(long interval)
+    {
+        return (int) (interval >>> 32);
+    }
+
+    private static int upper(long interval)
+    {
+        return (int) interval;
+    }
+
+
+    /**
+     * A mutable set of closed integer intervals, stored in normalized form (i.e. where overlapping intervals are
+     * converted to a single interval covering both). Thread-safe.
+     */
+    public static class Set
+    {
+        static long[] EMPTY = new long[0];
+
+        private volatile long[] ranges = EMPTY;
+
+        /**
+         * Adds an interval to the set, performing the necessary normalization.
+         */
+        public synchronized void add(int start, int end)
+        {
+            assert start <= end;
+            long[] ranges, newRanges;
+            {
+                ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+
+                // extend ourselves to cover any ranges we overlap
+                // record directly preceding our end may extend past us, so take the max of our end and its
+                int rpos = Arrays.binarySearch(ranges, ((end & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL); // floor (i.e. greatest <=) of the end position
+                if (rpos < 0)
+                    rpos = (-1 - rpos) - 1;
+                if (rpos >= 0)
+                {
+                    int extend = upper(ranges[rpos]);
+                    if (extend > end)
+                        end = extend;
+                }
+    
+                // record directly preceding our start may extend into us; if it does, we take it as our start
+                int lpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0); // lower (i.e. greatest <) of the start position
+                if (lpos < 0)
+                    lpos = -1 - lpos;
+                lpos -= 1;
+                if (lpos >= 0)
+                {
+                    if (upper(ranges[lpos]) >= start)
+                    {
+                        start = lower(ranges[lpos]);
+                        --lpos;
+                    }
+                }
+    
+                newRanges = new long[ranges.length - (rpos - lpos) + 1];
+                int dest = 0;
+                for (int i = 0; i <= lpos; ++i)
+                    newRanges[dest++] = ranges[i];
+                newRanges[dest++] = make(start, end);
+                for (int i = rpos + 1; i < ranges.length; ++i)
+                    newRanges[dest++] = ranges[i];
+            }
+            this.ranges = newRanges;
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(IntegerInterval iv)
+        {
+            long l = iv.interval;
+            return covers(lower(l), upper(l));
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(int start, int end)
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            int rpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL);        // floor (i.e. greatest <=) of the end position
+            if (rpos < 0)
+                rpos = (-1 - rpos) - 1;
+            if (rpos == -1)
+                return false;
+            return upper(ranges[rpos]) >= end;
+        }
+
+        /**
+         * Returns a lower bound for the whole set. Will throw if set is not empty.
+         */
+        public int lowerBound()
+        {
+            return lower(ranges[0]);
+        }
+
+        /**
+         * Returns an upper bound for the whole set. Will throw if set is not empty.
+         */
+        public int upperBound()
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            return upper(ranges[ranges.length - 1]);
+        }
+
+        public Collection<IntegerInterval> intervals()
+        {
+            return Lists.transform(Longs.asList(ranges), iv -> new IntegerInterval(iv));
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Arrays.hashCode(ranges);
+        }
+
+        @Override
+        public boolean equals(Object obj)
+        {
+            if (getClass() != obj.getClass())
+                return false;
+            Set other = (Set) obj;
+            return Arrays.equals(ranges, other.ranges);
+        }
+
+        public String toString()
+        {
+            return "[" + intervals().stream().map(IntegerInterval::toString).collect(Collectors.joining(", ")) + "]";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2df95c9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
new file mode 100644
index 0000000..c90b58d
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..76480b1
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+2048618157
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
new file mode 100644
index 0000000..3c716e9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
new file mode 100644
index 0000000..43beef3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..69c19f3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7027017
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..4b4078b
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+892998706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b2f5171
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..535d7df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..42876b0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8de00de
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..70a8c08
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1609623183
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..690d2e4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..d5bf83b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..5ff1f27
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7c47153
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..a6875fa
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1205036423
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b31055b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..3463560
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0b7faea
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
new file mode 100644
index 0000000..5862341
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..ee0485a
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+34605693
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
new file mode 100644
index 0000000..b3094bf
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
new file mode 100644
index 0000000..124f9a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..4a00428
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..36c7d92
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4017973941
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..ac35208
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0d9c077
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8aadb48
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..7bb4450
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1545836769
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..59e65ca
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c707d9e
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..6a5f57f
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..45b9e94
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4272819930
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c3299a0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index d517055..02b26c7 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -246,7 +246,7 @@ public class CommitLogStressTest
             verifySizes(commitLog);
 
             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
-                                               discardedPos);
+                    ReplayPosition.NONE, discardedPos);
             threads.clear();
             System.out.format("Discarded at %s\n", discardedPos);
             verifySizes(commitLog);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 e7b1ffa..d04ca9b 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -19,7 +19,9 @@ package org.apache.cassandra;
  *
  */
 
+import java.io.Closeable;
 import java.io.EOFException;
+import java.io.IOError;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -39,6 +41,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -57,7 +60,6 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -651,4 +653,21 @@ public class Util
             return queryStorage(cfs, orderGroup);
         }
     }
+
+    public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)
+    {
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+        return () -> BlacklistedDirectories.clearUnwritableUnsafe();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index a213edf..e3dc220 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -385,9 +385,9 @@ public abstract class CQLTester
     {
         try
         {
-            String currentTable = currentTable();
-            if (currentTable != null)
-                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceMajorCompaction();
+            ColumnFamilyStore store = getCurrentColumnFamilyStore();
+            if (store != null)
+                store.forceMajorCompaction();
         }
         catch (InterruptedException | ExecutionException e)
         {
@@ -397,9 +397,9 @@ public abstract class CQLTester
 
     public void cleanupCache()
     {
-        String currentTable = currentTable();
-        if (currentTable != null)
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
+        ColumnFamilyStore store = getCurrentColumnFamilyStore();
+        if (store != null)
+            store.cleanupCache();
     }
 
     public static FunctionName parseFunctionName(String qualifiedName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
index 1527b1e..26e7fe2 100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
@@ -19,18 +19,16 @@ package org.apache.cassandra.cql3;
 
 import static junit.framework.Assert.fail;
 
-import java.io.IOError;
+import java.io.Closeable;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.Keyspace;
@@ -48,12 +46,11 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableDie() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         KillerForTests killerForTests = new KillerForTests();
         JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
             flushAndExpectError();
@@ -71,10 +68,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableStop() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
             flushAndExpectError();
@@ -90,10 +86,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableIgnore() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
             flushAndExpectError();
@@ -104,7 +99,6 @@ public class OutOfSpaceTest extends CQLTester
         }
 
         // Next flush should succeed.
-        makeTable();
         flush();
     }
 
@@ -117,23 +111,6 @@ public class OutOfSpaceTest extends CQLTester
             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
     }
 
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
     public void flushAndExpectError() throws InterruptedException, ExecutionException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 39ba886..9a0ddb8 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -18,12 +18,7 @@
 */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
@@ -31,6 +26,7 @@ import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
@@ -46,18 +42,18 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.compress.DeflateCompressor;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -235,7 +231,7 @@ public class CommitLogTest
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 
         UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -262,7 +258,7 @@ public class CommitLogTest
         // "Flush": this won't delete anything
         UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
         CommitLog.instance.sync(true);
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid1, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
@@ -283,7 +279,7 @@ public class CommitLogTest
         // didn't write anything on cf1 since last flush (and we flush cf2)
 
         UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
@@ -494,8 +490,8 @@ public class CommitLogTest
             ReplayPosition position = CommitLog.instance.getContext();
             for (Keyspace ks : Keyspace.system())
                 for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
-                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
+                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, ReplayPosition.NONE, position);
+            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, ReplayPosition.NONE, position);
             assertEquals(1, CommitLog.instance.activeSegments());
         }
         finally
@@ -532,5 +528,138 @@ public class CommitLogTest
             DatabaseDescriptor.setAutoSnapshot(originalState);
         }
     }
+
+    @Test
+    public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+
+            for (int i = 0 ; i < 5 ; i++)
+            {
+                new RowUpdateBuilder(cfs.metadata, 0, "k")
+                    .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                    .build()
+                    .apply();
+
+                if (i == 2)
+                {
+                    try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                    {
+                        cfs.forceBlockingFlush();
+                    }
+                    catch (Throwable t)
+                    {
+                        // expected. Cause (after some wrappings) should be a write error
+                        while (!(t instanceof FSWriteError))
+                            t = t.getCause();
+                    }
+                }
+                else
+                    cfs.forceBlockingFlush();
+            }
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+        // If retries work subsequent flushes should clear up error and this should change to expect 0.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+            throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        for (int i = 0 ; i < 5 ; i++)
+        {
+            new RowUpdateBuilder(cfs.metadata, 0, "k")
+                .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                .build()
+                .apply();
+
+            Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+            if (i == 2)
+                current.makeUnflushable();
+
+            flushAction.accept(cfs, current);
+        }
+        if (performCompaction)
+            cfs.forceMajorCompaction();
+        // Make sure metadata saves and reads fine
+        for (SSTableReader reader : cfs.getLiveSSTables())
+            reader.reloadSSTableMetadata();
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+        // persisted all data in the commit log. Because we know there was an error, there must be something left to
+        // replay.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+    {
+        try
+        {
+            cfs.forceBlockingFlush();
+        }
+        catch (Throwable t)
+        {
+            // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+            while (!(t instanceof FSWriteError))
+                t = t.getCause();
+            // Wait for started flushes to complete.
+            cfs.switchMemtableIfCurrent(current);
+        }
+    };
+
+    BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+    {
+        // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+        // flushed data.
+        // This does not stop on errors and should retain segments for which flushing failed.
+        CommitLog.instance.forceRecycleAllSegments();
+
+        // Wait for started flushes to complete.
+        cfs.switchMemtableIfCurrent(current);
+    };
+
+    @Test
+    public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, false);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, false);
+    }
+
+    @Test
+    public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, true);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, true);
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
index 96e6ad6..cd1f295 100644
--- a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
@@ -80,9 +80,9 @@ public class NeverPurgeTest extends CQLTester
         execute("DELETE FROM %s WHERE a=3");
         cfs.forceBlockingFlush();
         cfs.enableAutoCompaction();
-        while (cfs.getSSTables().size() > 1)
+        while (cfs.getLiveSSTables().size() > 1)
             Thread.sleep(100);
-        verifyContainsTombstones(cfs.getSSTables(), 3);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 3);
     }
 
     private void testHelper(String deletionStatement) throws Throwable
@@ -94,7 +94,7 @@ public class NeverPurgeTest extends CQLTester
         Thread.sleep(1000);
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
-        verifyContainsTombstones(cfs.getSSTables(), 1);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 1);
     }
 
     private void verifyContainsTombstones(Collection<SSTableReader> sstables, int expectedTombstoneCount) throws Exception

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index b8de711..479e4e2 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -300,9 +300,6 @@ public class TrackerTest
         SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
         tracker.replaceFlushed(prev2, Collections.singleton(reader));
         Assert.assertEquals(1, tracker.getView().sstables.size());
-        Assert.assertEquals(1, tracker.getView().premature.size());
-        tracker.permitCompactionOfFlushed(singleton(reader));
-        Assert.assertEquals(0, tracker.getView().premature.size());
         Assert.assertEquals(1, listener.received.size());
         Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
         listener.received.clear();
@@ -318,13 +315,14 @@ public class TrackerTest
         tracker.markFlushing(prev1);
         reader = MockSchema.sstable(0, 10, true, cfs);
         cfs.invalidate(false);
-        tracker.replaceFlushed(prev1, Collections.singleton(reader));
-        tracker.permitCompactionOfFlushed(Collections.singleton(reader));
+        tracker.replaceFlushed(prev1, singleton(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, (((SSTableDeletingNotification) listener.received.get(0)).deleting));
-        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+        Assert.assertEquals(3, listener.received.size());
+        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
+        Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
+        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
         DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index a5dceca..7e4ced1 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -212,6 +212,6 @@ public class ViewTest
         for (int i = 0 ; i < sstableCount ; i++)
             sstables.add(MockSchema.sstable(i, cfs));
         return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
-                        Collections.<SSTableReader, SSTableReader>emptyMap(), Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+                        Collections.<SSTableReader, SSTableReader>emptyMap(), SSTableIntervalTree.build(sstables));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 62228e3..cefee52 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -71,7 +71,7 @@ public class LegacySSTableTest
      * See {@link #testGenerateSstables()} to generate sstables.
      * Take care on commit as you need to add the sstable files using {@code git add -f}
      */
-    public static final String[] legacyVersions = {"mb", "ma", "la", "ka", "jb"};
+    public static final String[] legacyVersions = {"mc", "mb", "ma", "la", "ka", "jb"};
 
     // 1200 chars
     static final String longString = "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 c842b7f..a7e822f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -781,7 +781,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
 
         cfs.addSSTable(writeFile(cfs, 1000));
 
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final Token firstToken = allSSTables.iterator().next().first.getToken();
         DatabaseDescriptor.setSSTablePreempiveOpenIntervalInMB(1);
@@ -880,7 +880,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
         truncate(cfs);
 
         cfs.addSSTable(writeFile(cfs, 100));
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final AtomicBoolean done = new AtomicBoolean(false);
         final AtomicBoolean failed = new AtomicBoolean(false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 93365ef..de12d57 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.io.sstable.Component;
@@ -85,8 +86,7 @@ public class MetadataSerializerTest
 
         CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                          .commitLogLowerBound(cllb)
-                                          .commitLogUpperBound(club);
+                                          .commitLogIntervals(new IntervalSet(cllb, club));
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
@@ -106,6 +106,18 @@ public class MetadataSerializerTest
         testOldReadsNew("ma", "mb");
     }
 
+    @Test
+    public void testMaReadMc() throws IOException
+    {
+        testOldReadsNew("ma", "mc");
+    }
+
+    @Test
+    public void testMbReadMc() throws IOException
+    {
+        testOldReadsNew("mb", "mc");
+    }
+
     public void testOldReadsNew(String oldV, String newV) throws IOException
     {
         Map<MetadataType, MetadataComponent> originalMetadata = constructMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
new file mode 100644
index 0000000..44843fd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
@@ -0,0 +1,326 @@
+package org.apache.cassandra.utils;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Futures;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.IntegerInterval.Set;
+
+public class IntegerIntervalsTest
+{
+    int[] values = new int[] { Integer.MIN_VALUE, -2, -1, 0, 5, 9, 13, Integer.MAX_VALUE };
+
+    @Test
+    public void testMake()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                assertEquals(values[i], iv.lower());
+                assertEquals(values[j], iv.upper());
+            }
+        }
+
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = 0; j < i; ++j)
+            {
+                try
+                {
+                    iv = new IntegerInterval(values[i], values[j]);
+                    fail("Assertion not thrown: " + values[i] + ", " + values[j]);
+                }
+                catch (AssertionError e)
+                {
+                    // expected
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverSingleThread()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                int k = 0;
+                for (; k < i; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[k], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < j; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < values.length; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[k], v.upper());
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 1000000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval interval = new IntegerInterval(0, 0);
+        int min = 0;
+        int max = 0;
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed).ints(streamSize).forEach(v -> interval.expandToCover(v));
+                return null;
+            });
+            min = Math.min(min, new Random(seed).ints(streamSize).min().getAsInt());
+            max = Math.max(max, new Random(seed).ints(streamSize).max().getAsInt());
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(min, interval.lower());
+        assertEquals(max, interval.upper());
+    }
+
+    void testSetAdd(int l, int r, Integer... expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertArrayEquals(expected, s
+                                    .intervals()
+                                    .stream()
+                                    .flatMap(x -> ImmutableList.of(x.lower(), x.upper()).stream())
+                                    .toArray());
+    }
+
+    void testSetAdd(int l, int r, String expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertEquals(expected, s.toString());
+    }
+
+    @Test
+    public void testSetAdd()
+    {
+        testSetAdd(Integer.MIN_VALUE, -4, Integer.MIN_VALUE, -4, -3, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -3, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -2, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -1, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 0, Integer.MIN_VALUE, 0, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 1, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 2, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 3, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, Integer.MAX_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE);
+
+        testSetAdd(-5, -4, "[[-5,-4], [-3,-1], [1,3]]");
+        testSetAdd(-5, -3, -5, -1, 1, 3);
+        testSetAdd(-5, -2, -5, -1, 1, 3);
+        testSetAdd(-5, -1, -5, -1, 1, 3);
+        testSetAdd(-5, 0, -5, 0, 1, 3);
+        testSetAdd(-5, 1, -5, 3);
+        testSetAdd(-5, 2, -5, 3);
+        testSetAdd(-5, 3, -5, 3);
+        testSetAdd(-5, 4, -5, 4);
+        testSetAdd(-5, Integer.MAX_VALUE, -5, Integer.MAX_VALUE);
+
+        testSetAdd(-3, -3, -3, -1, 1, 3);
+        testSetAdd(-3, -2, -3, -1, 1, 3);
+        testSetAdd(-3, -1, -3, -1, 1, 3);
+        testSetAdd(-3, 0, -3, 0, 1, 3);
+        testSetAdd(-3, 1, "[[-3,3]]");
+        testSetAdd(-3, 2, -3, 3);
+        testSetAdd(-3, 3, -3, 3);
+        testSetAdd(-3, 4, -3, 4);
+        testSetAdd(-3, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-2, -2, -3, -1, 1, 3);
+        testSetAdd(-2, -1, -3, -1, 1, 3);
+        testSetAdd(-2, 0, "[[-3,0], [1,3]]");
+        testSetAdd(-2, 1, -3, 3);
+        testSetAdd(-2, 2, -3, 3);
+        testSetAdd(-2, 3, -3, 3);
+        testSetAdd(-2, 4, -3, 4);
+        testSetAdd(-2, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-1, -1, -3, -1, 1, 3);
+        testSetAdd(-1, 0, -3, 0, 1, 3);
+        testSetAdd(-1, 1, -3, 3);
+        testSetAdd(-1, 2, -3, 3);
+        testSetAdd(-1, 3, -3, 3);
+        testSetAdd(-1, 4, -3, 4);
+        testSetAdd(-1, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(0, 0, -3, -1, 0, 0, 1, 3);
+        testSetAdd(0, 1, -3, -1, 0, 3);
+        testSetAdd(0, 2, -3, -1, 0, 3);
+        testSetAdd(0, 3, -3, -1, 0, 3);
+        testSetAdd(0, 4, -3, -1, 0, 4);
+        testSetAdd(0, Integer.MAX_VALUE, -3, -1, 0, Integer.MAX_VALUE);
+
+        testSetAdd(1, 1, -3, -1, 1, 3);
+        testSetAdd(1, 2, -3, -1, 1, 3);
+        testSetAdd(1, 3, -3, -1, 1, 3);
+        testSetAdd(1, 4, -3, -1, 1, 4);
+        testSetAdd(1, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(2, 2, -3, -1, 1, 3);
+        testSetAdd(2, 3, -3, -1, 1, 3);
+        testSetAdd(2, 4, -3, -1, 1, 4);
+        testSetAdd(2, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(3, 3, "[[-3,-1], [1,3]]");
+        testSetAdd(3, 4, -3, -1, 1, 4);
+        testSetAdd(3, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(4, 5, -3, -1, 1, 3, 4, 5);
+        testSetAdd(4, Integer.MAX_VALUE, -3, -1, 1, 3, 4, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testSetAddMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 10000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval.Set st = new IntegerInterval.Set();
+        final IntegerInterval.Set mt = new IntegerInterval.Set();
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed)
+                    .ints(streamSize)
+                    .forEach(v -> mt.add(v, v + 5));
+                return null;
+            });
+            new Random(seed)
+                .ints(streamSize)
+                .forEach(v -> st.add(v, v + 5));
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(st, mt);
+    }
+
+    void testSetCovers(int l, int r, boolean expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        assertEquals(expected, s.covers(new IntegerInterval(l, r)));
+    }
+
+
+    @Test
+    public void testSetCovers()
+    {
+        testSetCovers(Integer.MIN_VALUE, -4, false);
+        testSetCovers(Integer.MIN_VALUE, -3, false);
+        testSetCovers(Integer.MIN_VALUE, -2, false);
+        testSetCovers(Integer.MIN_VALUE, -1, false);
+        testSetCovers(Integer.MIN_VALUE, 0, false);
+        testSetCovers(Integer.MIN_VALUE, 1, false);
+        testSetCovers(Integer.MIN_VALUE, 2, false);
+        testSetCovers(Integer.MIN_VALUE, 3, false);
+        testSetCovers(Integer.MIN_VALUE, Integer.MAX_VALUE, false);
+
+        testSetCovers(-5, -4, false);
+        testSetCovers(-5, -3, false);
+        testSetCovers(-5, -2, false);
+        testSetCovers(-5, -1, false);
+        testSetCovers(-5, 0, false);
+        testSetCovers(-5, 1, false);
+        testSetCovers(-5, 2, false);
+        testSetCovers(-5, 3, false);
+        testSetCovers(-5, 4, false);
+        testSetCovers(-5, Integer.MAX_VALUE, false);
+
+        testSetCovers(-3, -3, true);
+        testSetCovers(-3, -2, true);
+        testSetCovers(-3, -1, true);
+        testSetCovers(-3, 0, false);
+        testSetCovers(-3, 1, false);
+        testSetCovers(-3, 2, false);
+        testSetCovers(-3, 3, false);
+        testSetCovers(-3, 4, false);
+        testSetCovers(-3, Integer.MAX_VALUE, false);
+
+        testSetCovers(-2, -2, true);
+        testSetCovers(-2, -1, true);
+        testSetCovers(-2, 0, false);
+        testSetCovers(-2, 1, false);
+        testSetCovers(-2, 2, false);
+        testSetCovers(-2, 3, false);
+        testSetCovers(-2, 4, false);
+        testSetCovers(-2, Integer.MAX_VALUE, false);
+
+        testSetCovers(-1, -1, true);
+        testSetCovers(-1, 0, false);
+        testSetCovers(-1, 1, false);
+        testSetCovers(-1, 2, false);
+        testSetCovers(-1, 3, false);
+        testSetCovers(-1, 4, false);
+        testSetCovers(-1, Integer.MAX_VALUE, false);
+
+        testSetCovers(0, 0, false);
+        testSetCovers(0, 1, false);
+        testSetCovers(0, 2, false);
+        testSetCovers(0, 3, false);
+        testSetCovers(0, 4, false);
+        testSetCovers(0, Integer.MAX_VALUE, false);
+
+        testSetCovers(1, 1, true);
+        testSetCovers(1, 2, true);
+        testSetCovers(1, 3, true);
+        testSetCovers(1, 4, false);
+        testSetCovers(1, Integer.MAX_VALUE, false);
+
+        testSetCovers(2, 2, true);
+        testSetCovers(2, 3, true);
+        testSetCovers(2, 4, false);
+        testSetCovers(2, Integer.MAX_VALUE, false);
+
+        testSetCovers(3, 3, true);
+        testSetCovers(3, 4, false);
+        testSetCovers(3, Integer.MAX_VALUE, false);
+
+        testSetCovers(4, 5, false);
+        testSetCovers(4, Integer.MAX_VALUE, false);
+    }
+}


[22/23] cassandra git commit: Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

Posted by sl...@apache.org.
Merge commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0' into cassandra-3.9

* commit '904cb5d10e0de1a6ca89249be8c257ed38a80ef0':
  Change commitlog and sstables to track dirty and clean intervals.
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-3.9
Commit: 7b1021733b55c8865f80e261697b4c079d086633
Parents: 21c92ca 904cb5d
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:39:15 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:39:56 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  70 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  21 +-
 .../AbstractCommitLogSegmentManager.java        |   4 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         | 105 ++----
 .../db/commitlog/CommitLogSegment.java          |  82 +++--
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  45 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  37 +--
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  12 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  37 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   3 +-
 test/unit/org/apache/cassandra/Util.java        |  21 +-
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 151 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 97 files changed, 1222 insertions(+), 369 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 289f370,b596fc9..43d28f3
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,10 -1,5 +1,11 @@@
 -3.0.9
 +3.9
 + * Fix nodetool tablestats miss SSTable count (CASSANDRA-12205)
 + * Fixed flacky SSTablesIteratedTest (CASSANDRA-12282)
 + * Fixed flacky SSTableRewriterTest: check file counts before calling validateCFS (CASSANDRA-12348)
 + * cqlsh: Fix handling of $$-escaped strings (CASSANDRA-12189)
 + * Fix SSL JMX requiring truststore containing server cert (CASSANDRA-12109)
 +Merged from 3.0:
+  * Change commitlog and sstables to track dirty and clean intervals (CASSANDRA-11828)
   * NullPointerException during compaction on table with static columns (CASSANDRA-12336)
   * Fixed ConcurrentModificationException when reading metrics in GraphiteReporter (CASSANDRA-11823)
   * Fix upgrade of super columns on thrift (CASSANDRA-12335)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/BlacklistedDirectories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 9d31b60,82604e2..21becfe
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -957,29 -922,19 +954,20 @@@ public class ColumnFamilyStore implemen
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
-         final CommitLogPosition commitLogUpperBound;
 -        volatile FSWriteError flushFailure = null;
 +        volatile Throwable flushFailure = null;
          final List<Memtable> memtables;
-         final List<Collection<SSTableReader>> readers;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier,
 +        private PostFlush(boolean flushSecondaryIndexes,
 +                          OpOrder.Barrier writeBarrier,
-                           CommitLogPosition commitLogUpperBound,
-                           List<Memtable> memtables,
-                           List<Collection<SSTableReader>> readers)
+                           List<Memtable> memtables)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
-             this.commitLogUpperBound = commitLogUpperBound;
              this.memtables = memtables;
-             this.readers = readers;
          }
  
 -        public ReplayPosition call()
 +        public CommitLogPosition call()
          {
-             if (discardFlushResults == ColumnFamilyStore.this)
-                 return commitLogUpperBound;
- 
              writeBarrier.await();
  
              /**
@@@ -1003,19 -958,13 +991,13 @@@
                  throw new IllegalStateException();
              }
  
-             // Must check commitLogUpperBound != null because Flush may find that all memtables are clean
-             // and so not set a commitLogUpperBound
 -            ReplayPosition commitLogUpperBound = ReplayPosition.NONE;
++            CommitLogPosition commitLogUpperBound = CommitLogPosition.NONE;
              // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
+             if (flushFailure == null && !memtables.isEmpty())
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     Collection<SSTableReader> reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyManager.replaceFlushed(memtable, reader);
-                 }
+                 Memtable memtable = memtables.get(0);
+                 commitLogUpperBound = memtable.getCommitLogUpperBound();
+                 CommitLog.instance.discardCompletedSegments(metadata.cfId, memtable.getCommitLogLowerBound(), commitLogUpperBound);
              }
  
              metric.pendingFlushes.dec();
@@@ -1079,9 -1027,9 +1060,9 @@@
  
              // we then issue the barrier; this lets us wait for all operations started prior to the barrier to complete;
              // since this happens after wiring up the commitLogUpperBound, we also know all operations with earlier
 -            // replay positions have also completed, i.e. the memtables are done and ready to flush
 +            // commit log segment position have also completed, i.e. the memtables are done and ready to flush
              writeBarrier.issue();
-             postFlush = new PostFlush(!truncate, writeBarrier, commitLogUpperBound.get(), memtables, readers);
+             postFlush = new PostFlush(!truncate, writeBarrier, memtables);
          }
  
          public void run()
@@@ -1111,110 -1059,23 +1092,108 @@@
              try
              {
                  for (Memtable memtable : memtables)
--                {
-                     this.readers.add(flushMemtable(memtable));
 -                    Collection<SSTableReader> readers = memtable.flush();
 -                    memtable.cfs.replaceFlushed(memtable, readers);
 -                    reclaim(memtable);
--                }
++                    flushMemtable(memtable);
              }
 -            catch (FSWriteError e)
 +            catch (Throwable t)
              {
 -                JVMStabilityInspector.inspectThrowable(e);
 -                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
 -                postFlush.flushFailure = e;
 +                JVMStabilityInspector.inspectThrowable(t);
 +                postFlush.flushFailure = t;
              }
 -
              // signal the post-flush we've done our work
              postFlush.latch.countDown();
          }
  
 +        public Collection<SSTableReader> flushMemtable(Memtable memtable)
 +        {
 +            List<Future<SSTableMultiWriter>> futures = new ArrayList<>();
 +            long totalBytesOnDisk = 0;
 +            long maxBytesOnDisk = 0;
 +            long minBytesOnDisk = Long.MAX_VALUE;
 +            List<SSTableReader> sstables = new ArrayList<>();
 +            try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.FLUSH))
 +            {
 +                List<Memtable.FlushRunnable> flushRunnables = null;
 +                List<SSTableMultiWriter> flushResults = null;
 +
 +                try
 +                {
 +                    // flush the memtable
 +                    flushRunnables = memtable.flushRunnables(txn);
 +
 +                    for (int i = 0; i < flushRunnables.size(); i++)
 +                        futures.add(perDiskflushExecutors[i].submit(flushRunnables.get(i)));
 +
 +                    flushResults = Lists.newArrayList(FBUtilities.waitOnFutures(futures));
 +                }
 +                catch (Throwable t)
 +                {
 +                    t = memtable.abortRunnables(flushRunnables, t);
 +                    t = txn.abort(t);
 +                    throw Throwables.propagate(t);
 +                }
 +
 +                try
 +                {
 +                    Iterator<SSTableMultiWriter> writerIterator = flushResults.iterator();
 +                    while (writerIterator.hasNext())
 +                    {
 +                        @SuppressWarnings("resource")
 +                        SSTableMultiWriter writer = writerIterator.next();
 +                        if (writer.getFilePointer() > 0)
 +                        {
 +                            writer.setOpenResult(true).prepareToCommit();
 +                        }
 +                        else
 +                        {
 +                            maybeFail(writer.abort(null));
 +                            writerIterator.remove();
 +                        }
 +                    }
 +                }
 +                catch (Throwable t)
 +                {
 +                    for (SSTableMultiWriter writer : flushResults)
 +                        t = writer.abort(t);
 +                    t = txn.abort(t);
 +                    Throwables.propagate(t);
 +                }
 +
 +                txn.prepareToCommit();
 +
 +                Throwable accumulate = null;
 +                for (SSTableMultiWriter writer : flushResults)
 +                    accumulate = writer.commit(accumulate);
 +
 +                maybeFail(txn.commit(accumulate));
 +
 +                for (SSTableMultiWriter writer : flushResults)
 +                {
 +                    Collection<SSTableReader> flushedSSTables = writer.finished();
 +                    for (SSTableReader sstable : flushedSSTables)
 +                    {
 +                        if (sstable != null)
 +                        {
 +                            sstables.add(sstable);
 +                            long size = sstable.bytesOnDisk();
 +                            totalBytesOnDisk += size;
 +                            maxBytesOnDisk = Math.max(maxBytesOnDisk, size);
 +                            minBytesOnDisk = Math.min(minBytesOnDisk, size);
 +                        }
 +                    }
 +                }
 +            }
-             memtable.cfs.data.replaceFlushed(memtable, sstables);
++            memtable.cfs.replaceFlushed(memtable, sstables);
 +            reclaim(memtable);
 +            memtable.cfs.compactionStrategyManager.compactionLogger.flush(sstables);
 +            logger.debug("Flushed to {} ({} sstables, {}), biggest {}, smallest {}",
 +                         sstables,
 +                         sstables.size(),
 +                         FBUtilities.prettyPrintMemory(totalBytesOnDisk),
 +                         FBUtilities.prettyPrintMemory(maxBytesOnDisk),
 +                         FBUtilities.prettyPrintMemory(minBytesOnDisk));
 +            return sstables;
 +        }
 +
          private void reclaim(final Memtable memtable)
          {
              // issue a read barrier for reclaiming the memory, and offload the wait to another thread
@@@ -2268,10 -2085,10 +2222,10 @@@
      {
          Callable<LifecycleTransaction> callable = new Callable<LifecycleTransaction>()
          {
 -            public LifecycleTransaction call() throws Exception
 +            public LifecycleTransaction call()
              {
                  assert data.getCompacting().isEmpty() : data.getCompacting();
-                 Iterable<SSTableReader> sstables = getPermittedToCompactSSTables();
+                 Iterable<SSTableReader> sstables = getLiveSSTables();
                  sstables = AbstractCompactionStrategy.filterSuspectSSTables(sstables);
                  sstables = ImmutableList.copyOf(sstables);
                  LifecycleTransaction modifier = data.tryModify(sstables, operationType);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Memtable.java
index 7a46d8a,3c77092..e9cca4a
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@@ -33,7 -33,9 +33,9 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.ColumnDefinition;
  import org.apache.cassandra.config.DatabaseDescriptor;
  import org.apache.cassandra.db.commitlog.CommitLog;
 +import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
 -import org.apache.cassandra.db.commitlog.ReplayPosition;
+ import org.apache.cassandra.db.compaction.OperationType;
  import org.apache.cassandra.db.filter.ClusteringIndexFilter;
  import org.apache.cassandra.db.filter.ColumnFilter;
  import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
@@@ -192,6 -194,11 +194,11 @@@ public class Memtable implements Compar
          return commitLogLowerBound.get();
      }
  
 -    public ReplayPosition getCommitLogUpperBound()
++    public CommitLogPosition getCommitLogUpperBound()
+     {
+         return commitLogUpperBound.get();
+     }
+ 
      public boolean isLive()
      {
          return allocator.isLive();
@@@ -361,63 -337,39 +368,72 @@@
          return minTimestamp;
      }
  
+     /**
+      * For testing only. Give this memtable too big a size to make it always fail flushing.
+      */
+     @VisibleForTesting
+     public void makeUnflushable()
+     {
+         liveDataSize.addAndGet(1L * 1024 * 1024 * 1024 * 1024 * 1024);
+     }
+ 
 -    private long estimatedSize()
 +    class FlushRunnable implements Callable<SSTableMultiWriter>
      {
 -        long keySize = 0;
 -        for (PartitionPosition key : partitions.keySet())
 +        private final long estimatedSize;
 +        private final ConcurrentNavigableMap<PartitionPosition, AtomicBTreePartition> toFlush;
 +
 +        private final boolean isBatchLogTable;
 +        private final SSTableMultiWriter writer;
 +
 +        // keeping these to be able to log what we are actually flushing
 +        private final PartitionPosition from;
 +        private final PartitionPosition to;
 +
 +        FlushRunnable(PartitionPosition from, PartitionPosition to, Directories.DataDirectory flushLocation, LifecycleTransaction txn)
          {
 -            //  make sure we don't write non-sensical keys
 -            assert key instanceof DecoratedKey;
 -            keySize += ((DecoratedKey)key).getKey().remaining();
 +            this(partitions.subMap(from, to), flushLocation, from, to, txn);
          }
 -        return (long) ((keySize // index entries
 -                        + keySize // keys in data file
 -                        + liveDataSize.get()) // data
 -                       * 1.2); // bloom filter and row index overhead
 -    }
  
 -    private Collection<SSTableReader> writeSortedContents(File sstableDirectory)
 -    {
 -        boolean isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SystemKeyspace.NAME);
 +        FlushRunnable(LifecycleTransaction txn)
 +        {
 +            this(partitions, null, null, null, txn);
 +        }
 +
 +        FlushRunnable(ConcurrentNavigableMap<PartitionPosition, AtomicBTreePartition> toFlush, Directories.DataDirectory flushLocation, PartitionPosition from, PartitionPosition to, LifecycleTransaction txn)
 +        {
 +            this.toFlush = toFlush;
 +            this.from = from;
 +            this.to = to;
 +            long keySize = 0;
 +            for (PartitionPosition key : toFlush.keySet())
 +            {
 +                //  make sure we don't write non-sensical keys
 +                assert key instanceof DecoratedKey;
 +                keySize += ((DecoratedKey) key).getKey().remaining();
 +            }
 +            estimatedSize = (long) ((keySize // index entries
 +                                    + keySize // keys in data file
 +                                    + liveDataSize.get()) // data
 +                                    * 1.2); // bloom filter and row index overhead
 +
 +            this.isBatchLogTable = cfs.name.equals(SystemKeyspace.BATCHES) && cfs.keyspace.getName().equals(SystemKeyspace.NAME);
  
 -        logger.debug("Writing {}", Memtable.this.toString());
 +            if (flushLocation == null)
 +                writer = createFlushWriter(txn, cfs.getSSTablePath(getDirectories().getWriteableLocationAsFile(estimatedSize)), columnsCollector.get(), statsCollector.get());
 +            else
 +                writer = createFlushWriter(txn, cfs.getSSTablePath(getDirectories().getLocationForDisk(flushLocation)), columnsCollector.get(), statsCollector.get());
 +
 +        }
  
 -        Collection<SSTableReader> ssTables;
 -        try (SSTableTxnWriter writer = createFlushWriter(cfs.getSSTablePath(sstableDirectory), columnsCollector.get(), statsCollector.get()))
 +        protected Directories getDirectories()
          {
 +            return cfs.getDirectories();
 +        }
 +
 +        private void writeSortedContents()
 +        {
 +            logger.debug("Writing {}, flushed range = ({}, {}]", Memtable.this.toString(), from, to);
 +
              boolean trackContention = logger.isTraceEnabled();
              int heavilyContendedRowCount = 0;
              // (we can't clear out the map as-we-go to free up memory,
@@@ -444,39 -396,58 +460,38 @@@
                  }
              }
  
 -            if (writer.getFilePointer() > 0)
 -            {
 -                logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
 -                                           writer.getFilename(),
 -                                           FBUtilities.prettyPrintMemory(writer.getFilePointer()),
 -                                           commitLogUpperBound));
 -
 -                // sstables should contain non-repaired data.
 -                ssTables = writer.finish(true);
 -            }
 -            else
 -            {
 -                logger.debug("Completed flushing {}; nothing needed to be retained.  Commitlog position was {}",
 -                             writer.getFilename(), commitLogUpperBound);
 -                writer.abort();
 -                ssTables = Collections.emptyList();
 -            }
 +            long bytesFlushed = writer.getFilePointer();
 +            logger.debug(String.format("Completed flushing %s (%s) for commitlog position %s",
 +                                                                              writer.getFilename(),
 +                                                                              FBUtilities.prettyPrintMemory(bytesFlushed),
 +                                                                              commitLogUpperBound));
 +            // Update the metrics
 +            cfs.metric.bytesFlushed.inc(bytesFlushed);
  
              if (heavilyContendedRowCount > 0)
 -                logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, partitions.size(), Memtable.this.toString()));
 -
 -            return ssTables;
 +                logger.trace(String.format("High update contention in %d/%d partitions of %s ", heavilyContendedRowCount, toFlush.size(), Memtable.this.toString()));
          }
 -    }
  
 -    @SuppressWarnings("resource") // log and writer closed by SSTableTxnWriter
 -    public SSTableTxnWriter createFlushWriter(String filename,
 -                                              PartitionColumns columns,
 -                                              EncodingStats stats)
 -    {
 -        // we operate "offline" here, as we expose the resulting reader consciously when done
 -        // (although we may want to modify this behaviour in future, to encapsulate full flush behaviour in LifecycleTransaction)
 -        LifecycleTransaction txn = null;
 -        try
 +        public SSTableMultiWriter createFlushWriter(LifecycleTransaction txn,
 +                                                  String filename,
 +                                                  PartitionColumns columns,
 +                                                  EncodingStats stats)
          {
 -            txn = LifecycleTransaction.offline(OperationType.FLUSH);
              MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator)
-                     .commitLogLowerBound(commitLogLowerBound.get())
-                     .commitLogUpperBound(commitLogUpperBound.get());
 -                    .commitLogIntervals(new IntervalSet(commitLogLowerBound.get(), commitLogUpperBound.get()));
 -
 -            return new SSTableTxnWriter(txn,
 -                                        cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),
 -                                                                     (long) partitions.size(),
 -                                                                     ActiveRepairService.UNREPAIRED_SSTABLE,
 -                                                                     sstableMetadataCollector,
 -                                                                     new SerializationHeader(true, cfs.metadata, columns, stats),
 -                                                                     txn));
++                    .commitLogIntervals(new IntervalSet<>(commitLogLowerBound.get(), commitLogUpperBound.get()));
++
 +            return cfs.createSSTableMultiWriter(Descriptor.fromFilename(filename),
 +                                                (long)toFlush.size(),
 +                                                ActiveRepairService.UNREPAIRED_SSTABLE,
 +                                                sstableMetadataCollector,
 +                                                new SerializationHeader(true, cfs.metadata, columns, stats), txn);
- 
          }
 -        catch (Throwable t)
 +
 +        @Override
 +        public SSTableMultiWriter call()
          {
 -            if (txn != null)
 -                txn.close();
 -            throw t;
 +            writeSortedContents();
 +            return writer;
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
index 7ea7439,0000000..8f3b7e4
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
@@@ -1,582 -1,0 +1,582 @@@
 +/*
 + * 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.commitlog;
 +
 +import java.io.File;
 +import java.io.IOException;
 +import java.util.*;
 +import java.util.concurrent.*;
 +import java.util.concurrent.atomic.AtomicLong;
 +
 +import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.collect.Iterables;
 +import com.google.common.util.concurrent.*;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.config.Schema;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.utils.*;
 +import org.apache.cassandra.utils.concurrent.WaitQueue;
 +
 +import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
 +
 +/**
 + * Performs eager-creation of commit log segments in a background thread. All the
 + * public methods are thread safe.
 + */
 +public abstract class AbstractCommitLogSegmentManager
 +{
 +    static final Logger logger = LoggerFactory.getLogger(AbstractCommitLogSegmentManager.class);
 +
 +    // Queue of work to be done by the manager thread, also used to wake the thread to perform segment allocation.
 +    private final BlockingQueue<Runnable> segmentManagementTasks = new LinkedBlockingQueue<>();
 +
 +    /** Segments that are ready to be used. Head of the queue is the one we allocate writes to */
 +    private final ConcurrentLinkedQueue<CommitLogSegment> availableSegments = new ConcurrentLinkedQueue<>();
 +
 +    /** Active segments, containing unflushed data */
 +    private final ConcurrentLinkedQueue<CommitLogSegment> activeSegments = new ConcurrentLinkedQueue<>();
 +
 +    /** The segment we are currently allocating commit log records to */
 +    protected volatile CommitLogSegment allocatingFrom = null;
 +
 +    private final WaitQueue hasAvailableSegments = new WaitQueue();
 +
 +    final String storageDirectory;
 +
 +    /**
 +     * Tracks commitlog size, in multiples of the segment size.  We need to do this so we can "promise" size
 +     * adjustments ahead of actually adding/freeing segments on disk, so that the "evict oldest segment" logic
 +     * can see the effect of recycling segments immediately (even though they're really happening asynchronously
 +     * on the manager thread, which will take a ms or two).
 +     */
 +    private final AtomicLong size = new AtomicLong();
 +
 +    /**
 +     * New segment creation is initially disabled because we'll typically get some "free" segments
 +     * recycled after log replay.
 +     */
 +    volatile boolean createReserveSegments = false;
 +
 +    private Thread managerThread;
 +    protected volatile boolean run = true;
 +    protected final CommitLog commitLog;
 +
 +    private static final SimpleCachedBufferPool bufferPool =
 +        new SimpleCachedBufferPool(DatabaseDescriptor.getCommitLogMaxCompressionBuffersInPool(), DatabaseDescriptor.getCommitLogSegmentSize());
 +
 +    AbstractCommitLogSegmentManager(final CommitLog commitLog, String storageDirectory)
 +    {
 +        this.commitLog = commitLog;
 +        this.storageDirectory = storageDirectory;
 +    }
 +
 +    void start()
 +    {
 +        // The run loop for the manager thread
 +        Runnable runnable = new WrappedRunnable()
 +        {
 +            public void runMayThrow() throws Exception
 +            {
 +                while (run)
 +                {
 +                    try
 +                    {
 +                        Runnable task = segmentManagementTasks.poll();
 +                        if (task == null)
 +                        {
 +                            // if we have no more work to do, check if we should create a new segment
 +                            if (!atSegmentLimit() &&
 +                                availableSegments.isEmpty() &&
 +                                (activeSegments.isEmpty() || createReserveSegments))
 +                            {
 +                                logger.trace("No segments in reserve; creating a fresh one");
 +                                // TODO : some error handling in case we fail to create a new segment
 +                                availableSegments.add(createSegment());
 +                                hasAvailableSegments.signalAll();
 +                            }
 +
 +                            // flush old Cfs if we're full
 +                            long unused = unusedCapacity();
 +                            if (unused < 0)
 +                            {
 +                                List<CommitLogSegment> segmentsToRecycle = new ArrayList<>();
 +                                long spaceToReclaim = 0;
 +                                for (CommitLogSegment segment : activeSegments)
 +                                {
 +                                    if (segment == allocatingFrom)
 +                                        break;
 +                                    segmentsToRecycle.add(segment);
 +                                    spaceToReclaim += DatabaseDescriptor.getCommitLogSegmentSize();
 +                                    if (spaceToReclaim + unused >= 0)
 +                                        break;
 +                                }
 +                                flushDataFrom(segmentsToRecycle, false);
 +                            }
 +
 +                            // Since we're operating on a "null" allocation task, block here for the next task on the
 +                            // queue rather than looping, grabbing another null, and repeating the above work.
 +                            try
 +                            {
 +                                task = segmentManagementTasks.take();
 +                            }
 +                            catch (InterruptedException e)
 +                            {
 +                                throw new AssertionError();
 +                            }
 +                        }
 +                        task.run();
 +                    }
 +                    catch (Throwable t)
 +                    {
 +                        JVMStabilityInspector.inspectThrowable(t);
 +                        if (!CommitLog.handleCommitError("Failed managing commit log segments", t))
 +                            return;
 +                        // sleep some arbitrary period to avoid spamming CL
 +                        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 +                    }
 +                }
 +            }
 +
 +            private boolean atSegmentLimit()
 +            {
 +                return CommitLogSegment.usesBufferPool(commitLog) && bufferPool.atLimit();
 +            }
 +        };
 +
 +        run = true;
 +
 +        managerThread = new Thread(runnable, "COMMIT-LOG-ALLOCATOR");
 +        managerThread.start();
 +    }
 +
 +
 +    /**
 +     * Shut down the CLSM. Used both during testing and during regular shutdown, so needs to stop everything.
 +     */
 +    public abstract void shutdown();
 +
 +    /**
 +     * Allocate a segment within this CLSM. Should either succeed or throw.
 +     */
 +    public abstract Allocation allocate(Mutation mutation, int size);
 +
 +    /**
 +     * The recovery and replay process replays mutations into memtables and flushes them to disk. Individual CLSM
 +     * decide what to do with those segments on disk after they've been replayed.
 +     */
 +    abstract void handleReplayedSegment(final File file);
 +
 +    /**
 +     * Hook to allow segment managers to track state surrounding creation of new segments. Onl perform as task submit
 +     * to segment manager so it's performed on segment management thread.
 +     */
 +    abstract CommitLogSegment createSegment();
 +
 +    /**
 +     * Indicates that a segment file has been flushed and is no longer needed. Only perform as task submit to segment
 +     * manager so it's performend on segment management thread, or perform while segment management thread is shutdown
 +     * during testing resets.
 +     *
 +     * @param segment segment to be discarded
 +     * @param delete  whether or not the segment is safe to be deleted.
 +     */
 +    abstract void discard(CommitLogSegment segment, boolean delete);
 +
 +
 +    /**
 +     * Grab the current CommitLogSegment we're allocating from. Also serves as a utility method to block while the allocator
 +     * is working on initial allocation of a CommitLogSegment.
 +     */
 +    CommitLogSegment allocatingFrom()
 +    {
 +        CommitLogSegment r = allocatingFrom;
 +        if (r == null)
 +        {
 +            advanceAllocatingFrom(null);
 +            r = allocatingFrom;
 +        }
 +        return r;
 +    }
 +
 +    /**
 +     * Fetches a new segment from the queue, signaling the management thread to create a new one if necessary, and "activates" it.
 +     * Blocks until a new segment is allocated and the thread requesting an advanceAllocatingFrom is signalled.
 +     *
 +     * WARNING: Assumes segment management thread always succeeds in allocating a new segment or kills the JVM.
 +     */
 +    protected void advanceAllocatingFrom(CommitLogSegment old)
 +    {
 +        while (true)
 +        {
 +            CommitLogSegment next;
 +            synchronized (this)
 +            {
 +                // do this in a critical section so we can atomically remove from availableSegments and add to allocatingFrom/activeSegments
 +                // see https://issues.apache.org/jira/browse/CASSANDRA-6557?focusedCommentId=13874432&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13874432
 +                if (allocatingFrom != old)
 +                    return;
 +                next = availableSegments.poll();
 +                if (next != null)
 +                {
 +                    allocatingFrom = next;
 +                    activeSegments.add(next);
 +                }
 +            }
 +
 +            if (next != null)
 +            {
 +                if (old != null)
 +                {
 +                    // Now we can run the user defined command just after switching to the new commit log.
 +                    // (Do this here instead of in the recycle call so we can get a head start on the archive.)
 +                    commitLog.archiver.maybeArchive(old);
 +
 +                    // ensure we don't continue to use the old file; not strictly necessary, but cleaner to enforce it
 +                    old.discardUnusedTail();
 +                }
 +
 +                // request that the CL be synced out-of-band, as we've finished a segment
 +                commitLog.requestExtraSync();
 +                return;
 +            }
 +
 +            // no more segments, so register to receive a signal when not empty
 +            WaitQueue.Signal signal = hasAvailableSegments.register(commitLog.metrics.waitingOnSegmentAllocation.time());
 +
 +            // trigger the management thread; this must occur after registering
 +            // the signal to ensure we are woken by any new segment creation
 +            wakeManager();
 +
 +            // check if the queue has already been added to before waiting on the signal, to catch modifications
 +            // that happened prior to registering the signal; *then* check to see if we've been beaten to making the change
 +            if (!availableSegments.isEmpty() || allocatingFrom != old)
 +            {
 +                signal.cancel();
 +                // if we've been beaten, just stop immediately
 +                if (allocatingFrom != old)
 +                    return;
 +                // otherwise try again, as there should be an available segment
 +                continue;
 +            }
 +
 +            // can only reach here if the queue hasn't been inserted into
 +            // before we registered the signal, as we only remove items from the queue
 +            // after updating allocatingFrom. Can safely block until we are signalled
 +            // by the allocator that new segments have been published
 +            signal.awaitUninterruptibly();
 +        }
 +    }
 +
 +    protected void wakeManager()
 +    {
 +        // put a NO-OP on the queue, to trigger management thread (and create a new segment if necessary)
 +        segmentManagementTasks.add(Runnables.doNothing());
 +    }
 +
 +    /**
 +     * Switch to a new segment, regardless of how much is left in the current one.
 +     *
 +     * Flushes any dirty CFs for this segment and any older segments, and then recycles
 +     * the segments
 +     */
 +    void forceRecycleAll(Iterable<UUID> droppedCfs)
 +    {
 +        List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
 +        CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1);
 +        advanceAllocatingFrom(last);
 +
 +        // wait for the commit log modifications
 +        last.waitForModifications();
 +
 +        // make sure the writes have materialized inside of the memtables by waiting for all outstanding writes
 +        // on the relevant keyspaces to complete
 +        Keyspace.writeOrder.awaitNewBarrier();
 +
 +        // flush and wait for all CFs that are dirty in segments up-to and including 'last'
 +        Future<?> future = flushDataFrom(segmentsToRecycle, true);
 +        try
 +        {
 +            future.get();
 +
 +            for (CommitLogSegment segment : activeSegments)
 +                for (UUID cfId : droppedCfs)
-                     segment.markClean(cfId, segment.getCurrentCommitLogPosition());
++                    segment.markClean(cfId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
 +
 +            // now recycle segments that are unused, as we may not have triggered a discardCompletedSegments()
 +            // if the previous active segment was the only one to recycle (since an active segment isn't
 +            // necessarily dirty, and we only call dCS after a flush).
 +            for (CommitLogSegment segment : activeSegments)
 +            {
 +                if (segment.isUnused())
 +                    recycleSegment(segment);
 +            }
 +
 +            CommitLogSegment first;
 +            if ((first = activeSegments.peek()) != null && first.id <= last.id)
 +                logger.error("Failed to force-recycle all segments; at least one segment is still in use with dirty CFs.");
 +        }
 +        catch (Throwable t)
 +        {
 +            // for now just log the error
 +            logger.error("Failed waiting for a forced recycle of in-use commit log segments", t);
 +        }
 +    }
 +
 +    /**
 +     * Indicates that a segment is no longer in use and that it should be recycled.
 +     *
 +     * @param segment segment that is no longer in use
 +     */
 +    void recycleSegment(final CommitLogSegment segment)
 +    {
 +        boolean archiveSuccess = commitLog.archiver.maybeWaitForArchiving(segment.getName());
 +        if (activeSegments.remove(segment))
 +        {
 +            // if archiving (command) was not successful then leave the file alone. don't delete or recycle.
 +            discardSegment(segment, archiveSuccess);
 +        }
 +        else
 +        {
 +            logger.warn("segment {} not found in activeSegments queue", segment);
 +        }
 +    }
 +
 +    /**
 +     * Indicates that a segment file should be deleted.
 +     *
 +     * @param segment segment to be discarded
 +     */
 +    private void discardSegment(final CommitLogSegment segment, final boolean deleteFile)
 +    {
 +        logger.trace("Segment {} is no longer active and will be deleted {}", segment, deleteFile ? "now" : "by the archive script");
 +        segmentManagementTasks.add(() -> discard(segment, deleteFile));
 +    }
 +
 +    /**
 +     * Adjust the tracked on-disk size. Called by individual segments to reflect writes, allocations and discards.
 +     * @param addedSize
 +     */
 +    void addSize(long addedSize)
 +    {
 +        size.addAndGet(addedSize);
 +    }
 +
 +    /**
 +     * @return the space (in bytes) used by all segment files.
 +     */
 +    public long onDiskSize()
 +    {
 +        return size.get();
 +    }
 +
 +    private long unusedCapacity()
 +    {
 +        long total = DatabaseDescriptor.getTotalCommitlogSpaceInMB() * 1024 * 1024;
 +        long currentSize = size.get();
 +        logger.trace("Total active commitlog segment space used is {} out of {}", currentSize, total);
 +        return total - currentSize;
 +    }
 +
 +    /**
 +     * @param name the filename to check
 +     * @return true if file is managed by this manager.
 +     */
 +    public boolean manages(String name)
 +    {
 +        for (CommitLogSegment segment : Iterables.concat(activeSegments, availableSegments))
 +            if (segment.getName().equals(name))
 +                return true;
 +        return false;
 +    }
 +
 +    /**
 +     * Throws a flag that enables the behavior of keeping at least one spare segment
 +     * available at all times.
 +     */
 +    void enableReserveSegmentCreation()
 +    {
 +        createReserveSegments = true;
 +        wakeManager();
 +    }
 +
 +    /**
 +     * Force a flush on all CFs that are still dirty in @param segments.
 +     *
 +     * @return a Future that will finish when all the flushes are complete.
 +     */
 +    private Future<?> flushDataFrom(List<CommitLogSegment> segments, boolean force)
 +    {
 +        if (segments.isEmpty())
 +            return Futures.immediateFuture(null);
 +        final CommitLogPosition maxCommitLogPosition = segments.get(segments.size() - 1).getCurrentCommitLogPosition();
 +
 +        // a map of CfId -> forceFlush() to ensure we only queue one flush per cf
 +        final Map<UUID, ListenableFuture<?>> flushes = new LinkedHashMap<>();
 +
 +        for (CommitLogSegment segment : segments)
 +        {
 +            for (UUID dirtyCFId : segment.getDirtyCFIDs())
 +            {
 +                Pair<String,String> pair = Schema.instance.getCF(dirtyCFId);
 +                if (pair == null)
 +                {
 +                    // even though we remove the schema entry before a final flush when dropping a CF,
 +                    // it's still possible for a writer to race and finish his append after the flush.
 +                    logger.trace("Marking clean CF {} that doesn't exist anymore", dirtyCFId);
-                     segment.markClean(dirtyCFId, segment.getCurrentCommitLogPosition());
++                    segment.markClean(dirtyCFId, CommitLogPosition.NONE, segment.getCurrentCommitLogPosition());
 +                }
 +                else if (!flushes.containsKey(dirtyCFId))
 +                {
 +                    String keyspace = pair.left;
 +                    final ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(dirtyCFId);
 +                    // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush,
 +                    // no deadlock possibility since switchLock removal
 +                    flushes.put(dirtyCFId, force ? cfs.forceFlush() : cfs.forceFlush(maxCommitLogPosition));
 +                }
 +            }
 +        }
 +
 +        return Futures.allAsList(flushes.values());
 +    }
 +
 +    /**
 +     * Stops CL, for testing purposes. DO NOT USE THIS OUTSIDE OF TESTS.
 +     * Only call this after the AbstractCommitLogService is shut down.
 +     */
 +    public void stopUnsafe(boolean deleteSegments)
 +    {
 +        logger.trace("CLSM closing and clearing existing commit log segments...");
 +        createReserveSegments = false;
 +
 +        awaitManagementTasksCompletion();
 +
 +        shutdown();
 +        try
 +        {
 +            awaitTermination();
 +        }
 +        catch (InterruptedException e)
 +        {
 +            throw new RuntimeException(e);
 +        }
 +
 +        synchronized (this)
 +        {
 +            for (CommitLogSegment segment : activeSegments)
 +                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
 +            activeSegments.clear();
 +
 +            for (CommitLogSegment segment : availableSegments)
 +                closeAndDeleteSegmentUnsafe(segment, deleteSegments);
 +            availableSegments.clear();
 +        }
 +
 +        allocatingFrom = null;
 +
 +        segmentManagementTasks.clear();
 +
 +        size.set(0L);
 +
 +        logger.trace("CLSM done with closing and clearing existing commit log segments.");
 +    }
 +
 +    // Used by tests only.
 +    void awaitManagementTasksCompletion()
 +    {
 +        while (!segmentManagementTasks.isEmpty())
 +            Thread.yield();
 +        // The last management task is not yet complete. Wait a while for it.
 +        Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
 +        // TODO: If this functionality is required by anything other than tests, signalling must be used to ensure
 +        // waiting completes correctly.
 +    }
 +
 +    /**
 +     * Explicitly for use only during resets in unit testing.
 +     */
 +    private void closeAndDeleteSegmentUnsafe(CommitLogSegment segment, boolean delete)
 +    {
 +        try
 +        {
 +            discard(segment, delete);
 +        }
 +        catch (AssertionError ignored)
 +        {
 +            // segment file does not exist
 +        }
 +    }
 +
 +    /**
 +     * Returns when the management thread terminates.
 +     */
 +    public void awaitTermination() throws InterruptedException
 +    {
 +        managerThread.join();
 +
 +        for (CommitLogSegment segment : activeSegments)
 +            segment.close();
 +
 +        for (CommitLogSegment segment : availableSegments)
 +            segment.close();
 +
 +        bufferPool.shutdown();
 +    }
 +
 +    /**
 +     * @return a read-only collection of the active commit log segments
 +     */
 +    @VisibleForTesting
 +    public Collection<CommitLogSegment> getActiveSegments()
 +    {
 +        return Collections.unmodifiableCollection(activeSegments);
 +    }
 +
 +    /**
 +     * @return the current CommitLogPosition of the active segment we're allocating from
 +     */
 +    CommitLogPosition getCurrentPosition()
 +    {
 +        return allocatingFrom().getCurrentCommitLogPosition();
 +    }
 +
 +    /**
 +     * Forces a disk flush on the commit log files that need it.  Blocking.
 +     */
 +    public void sync(boolean syncAllSegments) throws IOException
 +    {
 +        CommitLogSegment current = allocatingFrom();
 +        for (CommitLogSegment segment : getActiveSegments())
 +        {
 +            if (!syncAllSegments && segment.id > current.id)
 +                return;
 +            segment.sync();
 +        }
 +    }
 +
 +    /**
 +     * Used by compressed and encrypted segments to share a buffer pool across the CLSM.
 +     */
 +    SimpleCachedBufferPool getBufferPool()
 +    {
 +        return bufferPool;
 +    }
 +}
 +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index b66221c,dfe3f91..32f69eb
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@@ -298,20 -290,21 +298,21 @@@ public class CommitLog implements Commi
       * given. Discards any commit log segments that are no longer used.
       *
       * @param cfId    the column family ID that was flushed
-      * @param context the commit log segment position of the flush
+      * @param lowerBound the lowest covered replay position of the flush
+      * @param lowerBound the highest covered replay position of the flush
       */
-     public void discardCompletedSegments(final UUID cfId, final CommitLogPosition context)
 -    public void discardCompletedSegments(final UUID cfId, final ReplayPosition lowerBound, final ReplayPosition upperBound)
++    public void discardCompletedSegments(final UUID cfId, final CommitLogPosition lowerBound, final CommitLogPosition upperBound)
      {
-         logger.trace("discard completed log segments for {}, table {}", context, cfId);
+         logger.trace("discard completed log segments for {}-{}, table {}", lowerBound, upperBound, cfId);
  
          // Go thru the active segment files, which are ordered oldest to newest, marking the
 -        // flushed CF as clean, until we reach the segment file containing the ReplayPosition passed
 +        // flushed CF as clean, until we reach the segment file containing the CommitLogPosition passed
          // in the arguments. Any segments that become unused after they are marked clean will be
          // recycled or discarded.
 -        for (Iterator<CommitLogSegment> iter = allocator.getActiveSegments().iterator(); iter.hasNext();)
 +        for (Iterator<CommitLogSegment> iter = segmentManager.getActiveSegments().iterator(); iter.hasNext();)
          {
              CommitLogSegment segment = iter.next();
-             segment.markClean(cfId, context);
+             segment.markClean(cfId, lowerBound, upperBound);
  
              if (segment.isUnused())
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index c8e597f,af8efb4..92364c8
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@@ -54,32 -70,31 +54,32 @@@ public class CommitLogReplayer implemen
      static final String IGNORE_REPLAY_ERRORS_PROPERTY = "cassandra.commitlog.ignorereplayerrors";
      private static final Logger logger = LoggerFactory.getLogger(CommitLogReplayer.class);
      private static final int MAX_OUTSTANDING_REPLAY_COUNT = Integer.getInteger("cassandra.commitlog_max_outstanding_replay_count", 1024);
 -    private static final int LEGACY_END_OF_SEGMENT_MARKER = 0;
  
 -    private final Set<Keyspace> keyspacesRecovered;
 -    private final List<Future<?>> futures;
 -    private final Map<UUID, AtomicInteger> invalidMutations;
 +    private final Set<Keyspace> keyspacesReplayed;
 +    private final Queue<Future<Integer>> futures;
 +
      private final AtomicInteger replayedCount;
-     private final Map<UUID, ReplayPositionFilter> cfPersisted;
 -    private final Map<UUID, IntervalSet<ReplayPosition>> cfPersisted;
 -    private final ReplayPosition globalPosition;
 -    private final CRC32 checksum;
 -    private byte[] buffer;
 -    private byte[] uncompressedBuffer;
++    private final Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted;
 +    private final CommitLogPosition globalPosition;
 +
 +    // Used to throttle speed of replay of mutations if we pass the max outstanding count
 +    private long pendingMutationBytes = 0;
  
      private final ReplayFilter replayFilter;
      private final CommitLogArchiver archiver;
  
 -    CommitLogReplayer(CommitLog commitLog, ReplayPosition globalPosition, Map<UUID, IntervalSet<ReplayPosition>> cfPersisted, ReplayFilter replayFilter)
 +    @VisibleForTesting
 +    protected CommitLogReader commitLogReader;
 +
 +    CommitLogReplayer(CommitLog commitLog,
 +                      CommitLogPosition globalPosition,
-                       Map<UUID, ReplayPositionFilter> cfPersisted,
++                      Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted,
 +                      ReplayFilter replayFilter)
      {
 -        this.keyspacesRecovered = new NonBlockingHashSet<Keyspace>();
 -        this.futures = new ArrayList<Future<?>>();
 -        this.buffer = new byte[4096];
 -        this.uncompressedBuffer = new byte[4096];
 -        this.invalidMutations = new HashMap<UUID, AtomicInteger>();
 +        this.keyspacesReplayed = new NonBlockingHashSet<Keyspace>();
 +        this.futures = new ArrayDeque<Future<Integer>>();
          // count the number of replayed mutation. We don't really care about atomicity, but we need it to be a reference.
          this.replayedCount = new AtomicInteger();
 -        this.checksum = new CRC32();
          this.cfPersisted = cfPersisted;
          this.globalPosition = globalPosition;
          this.replayFilter = replayFilter;
@@@ -89,10 -103,9 +89,10 @@@
  
      public static CommitLogReplayer construct(CommitLog commitLog)
      {
-         // compute per-CF and global commit log segment positions
-         Map<UUID, ReplayPositionFilter> cfPersisted = new HashMap<>();
+         // compute per-CF and global replay intervals
 -        Map<UUID, IntervalSet<ReplayPosition>> cfPersisted = new HashMap<>();
++        Map<UUID, IntervalSet<CommitLogPosition>> cfPersisted = new HashMap<>();
          ReplayFilter replayFilter = ReplayFilter.create();
-         CommitLogPosition globalPosition = null;
++
          for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
          {
              // but, if we've truncated the cf in question, then we need to need to start replay after the truncation
@@@ -117,15 -130,11 +117,11 @@@
                  }
              }
  
-             ReplayPositionFilter filter = new ReplayPositionFilter(cfs.getSSTables(), truncatedAt);
-             if (!filter.isEmpty())
-                 cfPersisted.put(cfs.metadata.cfId, filter);
-             else
-                 globalPosition = CommitLogPosition.NONE; // if we have no ranges for this CF, we must replay everything and filter
 -            IntervalSet<ReplayPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
++            IntervalSet<CommitLogPosition> filter = persistedIntervals(cfs.getLiveSSTables(), truncatedAt);
+             cfPersisted.put(cfs.metadata.cfId, filter);
          }
-         if (globalPosition == null)
-             globalPosition = firstNotCovered(cfPersisted.values());
-         logger.trace("Global commit log segment position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
 -        ReplayPosition globalPosition = firstNotCovered(cfPersisted.values());
++        CommitLogPosition globalPosition = firstNotCovered(cfPersisted.values());
+         logger.debug("Global replay position is {} from columnfamilies {}", globalPosition, FBUtilities.toString(cfPersisted));
          return new CommitLogReplayer(commitLog, globalPosition, cfPersisted, replayFilter);
      }
  
@@@ -174,135 -208,38 +170,105 @@@
          return replayedCount.get();
      }
  
 -    private int readSyncMarker(CommitLogDescriptor descriptor, int offset, RandomAccessReader reader, boolean tolerateTruncation) throws IOException
 +    /*
 +     * Wrapper around initiating mutations read from the log to make it possible
 +     * to spy on initiated mutations for test
 +     */
 +    @VisibleForTesting
 +    public static class MutationInitiator
      {
 -        if (offset > reader.length() - CommitLogSegment.SYNC_MARKER_SIZE)
 -        {
 -            // There was no room in the segment to write a final header. No data could be present here.
 -            return -1;
 -        }
 -        reader.seek(offset);
 -        CRC32 crc = new CRC32();
 -        updateChecksumInt(crc, (int) (descriptor.id & 0xFFFFFFFFL));
 -        updateChecksumInt(crc, (int) (descriptor.id >>> 32));
 -        updateChecksumInt(crc, (int) reader.getPosition());
 -        int end = reader.readInt();
 -        long filecrc = reader.readInt() & 0xffffffffL;
 -        if (crc.getValue() != filecrc)
 +        protected Future<Integer> initiateMutation(final Mutation mutation,
 +                                                   final long segmentId,
 +                                                   final int serializedSize,
 +                                                   final int entryLocation,
 +                                                   final CommitLogReplayer commitLogReplayer)
          {
 -            if (end != 0 || filecrc != 0)
 +            Runnable runnable = new WrappedRunnable()
              {
 -                handleReplayError(false,
 -                                  "Encountered bad header at position %d of commit log %s, with invalid CRC. " +
 -                                  "The end of segment marker should be zero.",
 -                                  offset, reader.getPath());
 -            }
 -            return -1;
 -        }
 -        else if (end < offset || end > reader.length())
 -        {
 -            handleReplayError(tolerateTruncation, "Encountered bad header at position %d of commit log %s, with bad position but valid CRC",
 -                              offset, reader.getPath());
 -            return -1;
 +                public void runMayThrow()
 +                {
 +                    if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
 +                        return;
 +                    if (commitLogReplayer.pointInTimeExceeded(mutation))
 +                        return;
 +
 +                    final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
 +
 +                    // Rebuild the mutation, omitting column families that
 +                    //    a) the user has requested that we ignore,
 +                    //    b) have already been flushed,
 +                    // or c) are part of a cf that was dropped.
 +                    // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
 +                    Mutation newMutation = null;
 +                    for (PartitionUpdate update : commitLogReplayer.replayFilter.filter(mutation))
 +                    {
 +                        if (Schema.instance.getCF(update.metadata().cfId) == null)
 +                            continue; // dropped
 +
 +                        // replay if current segment is newer than last flushed one or,
 +                        // if it is the last known segment, if we are after the commit log segment position
 +                        if (commitLogReplayer.shouldReplay(update.metadata().cfId, new CommitLogPosition(segmentId, entryLocation)))
 +                        {
 +                            if (newMutation == null)
 +                                newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
 +                            newMutation.add(update);
 +                            commitLogReplayer.replayedCount.incrementAndGet();
 +                        }
 +                    }
 +                    if (newMutation != null)
 +                    {
 +                        assert !newMutation.isEmpty();
 +
 +                        try
 +                        {
 +                            Uninterruptibles.getUninterruptibly(Keyspace.open(newMutation.getKeyspaceName()).applyFromCommitLog(newMutation));
 +                        }
 +                        catch (ExecutionException e)
 +                        {
 +                            throw Throwables.propagate(e.getCause());
 +                        }
 +
 +                        commitLogReplayer.keyspacesReplayed.add(keyspace);
 +                    }
 +                }
 +            };
 +            return StageManager.getStage(Stage.MUTATION).submit(runnable, serializedSize);
          }
 -        return end;
 +    }
 +
 +    /**
-      * A filter of known safe-to-discard commit log replay positions, based on
++     * A set of known safe-to-discard commit log replay positions, based on
 +     * the range covered by on disk sstables and those prior to the most recent truncation record
 +     */
-     public static class ReplayPositionFilter
++    public static IntervalSet<CommitLogPosition> persistedIntervals(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
 +    {
-         final NavigableMap<CommitLogPosition, CommitLogPosition> persisted = new TreeMap<>();
-         public ReplayPositionFilter(Iterable<SSTableReader> onDisk, CommitLogPosition truncatedAt)
-         {
-             for (SSTableReader reader : onDisk)
-             {
-                 CommitLogPosition start = reader.getSSTableMetadata().commitLogLowerBound;
-                 CommitLogPosition end = reader.getSSTableMetadata().commitLogUpperBound;
-                 add(persisted, start, end);
-             }
-             if (truncatedAt != null)
-                 add(persisted, CommitLogPosition.NONE, truncatedAt);
-         }
++        IntervalSet.Builder<CommitLogPosition> builder = new IntervalSet.Builder<>();
++        for (SSTableReader reader : onDisk)
++            builder.addAll(reader.getSSTableMetadata().commitLogIntervals);
 +
-         private static void add(NavigableMap<CommitLogPosition, CommitLogPosition> ranges, CommitLogPosition start, CommitLogPosition end)
-         {
-             // extend ourselves to cover any ranges we overlap
-             // record directly preceding our end may extend past us, so take the max of our end and its
-             Map.Entry<CommitLogPosition, CommitLogPosition> extend = ranges.floorEntry(end);
-             if (extend != null && extend.getValue().compareTo(end) > 0)
-                 end = extend.getValue();
- 
-             // record directly preceding our start may extend into us; if it does, we take it as our start
-             extend = ranges.lowerEntry(start);
-             if (extend != null && extend.getValue().compareTo(start) >= 0)
-                 start = extend.getKey();
- 
-             ranges.subMap(start, end).clear();
-             ranges.put(start, end);
-         }
- 
-         public boolean shouldReplay(CommitLogPosition position)
-         {
-             // replay ranges are start exclusive, end inclusive
-             Map.Entry<CommitLogPosition, CommitLogPosition> range = persisted.lowerEntry(position);
-             return range == null || position.compareTo(range.getValue()) > 0;
-         }
- 
-         public boolean isEmpty()
-         {
-             return persisted.isEmpty();
-         }
++        if (truncatedAt != null)
++            builder.add(CommitLogPosition.NONE, truncatedAt);
++        return builder.build();
 +    }
 +
-     public static CommitLogPosition firstNotCovered(Iterable<ReplayPositionFilter> ranges)
++    /**
++     * Find the earliest commit log position that is not covered by the known flushed ranges for some table.
++     *
++     * For efficiency this assumes that the first contiguously flushed interval we know of contains the moment that the
++     * given table was constructed* and hence we can start replay from the end of that interval.
++     *
++     * If such an interval is not known, we must replay from the beginning.
++     *
++     * * This is not true only until if the very first flush of a table stalled or failed, while the second or latter
++     *   succeeded. The chances of this happening are at most very low, and if the assumption does prove to be
++     *   incorrect during replay there is little chance that the affected deployment is in production.
++     */
++    public static CommitLogPosition firstNotCovered(Collection<IntervalSet<CommitLogPosition>> ranges)
 +    {
-         CommitLogPosition min = null;
-         for (ReplayPositionFilter map : ranges)
-         {
-             CommitLogPosition first = map.persisted.firstEntry().getValue();
-             if (min == null)
-                 min = first;
-             else
-                 min = Ordering.natural().min(min, first);
-         }
-         if (min == null)
-             return CommitLogPosition.NONE;
-         return min;
++        return ranges.stream()
++                .map(intervals -> Iterables.getFirst(intervals.ends(), CommitLogPosition.NONE)) 
++                .min(Ordering.natural())
++                .get(); // iteration is per known-CF, there must be at least one. 
      }
  
      abstract static class ReplayFilter
@@@ -386,12 -323,346 +352,11 @@@
       *
       * @return true iff replay is necessary
       */
 -    private boolean shouldReplay(UUID cfId, ReplayPosition position)
 +    private boolean shouldReplay(UUID cfId, CommitLogPosition position)
      {
-         ReplayPositionFilter filter = cfPersisted.get(cfId);
-         return filter == null || filter.shouldReplay(position);
+         return !cfPersisted.get(cfId).contains(position);
      }
  
 -    @SuppressWarnings("resource")
 -    public void recover(File file, boolean tolerateTruncation) throws IOException
 -    {
 -        CommitLogDescriptor desc = CommitLogDescriptor.fromFileName(file.getName());
 -        try(ChannelProxy channel = new ChannelProxy(file);
 -            RandomAccessReader reader = RandomAccessReader.open(channel))
 -        {
 -            if (desc.version < CommitLogDescriptor.VERSION_21)
 -            {
 -                if (logAndCheckIfShouldSkip(file, desc))
 -                    return;
 -                if (globalPosition.segment == desc.id)
 -                    reader.seek(globalPosition.position);
 -                replaySyncSection(reader, (int) reader.length(), desc, desc.fileName(), tolerateTruncation);
 -                return;
 -            }
 -
 -            final long segmentId = desc.id;
 -            try
 -            {
 -                desc = CommitLogDescriptor.readHeader(reader);
 -            }
 -            catch (IOException e)
 -            {
 -                desc = null;
 -            }
 -            if (desc == null) {
 -                handleReplayError(false, "Could not read commit log descriptor in file %s", file);
 -                return;
 -            }
 -            if (segmentId != desc.id)
 -            {
 -                handleReplayError(false, "Segment id mismatch (filename %d, descriptor %d) in file %s", segmentId, desc.id, file);
 -                // continue processing if ignored.
 -            }
 -
 -            if (logAndCheckIfShouldSkip(file, desc))
 -                return;
 -
 -            ICompressor compressor = null;
 -            if (desc.compression != null)
 -            {
 -                try
 -                {
 -                    compressor = CompressionParams.createCompressor(desc.compression);
 -                }
 -                catch (ConfigurationException e)
 -                {
 -                    handleReplayError(false, "Unknown compression: %s", e.getMessage());
 -                    return;
 -                }
 -            }
 -
 -            assert reader.length() <= Integer.MAX_VALUE;
 -            int end = (int) reader.getFilePointer();
 -            int replayEnd = end;
 -
 -            while ((end = readSyncMarker(desc, end, reader, tolerateTruncation)) >= 0)
 -            {
 -                int replayPos = replayEnd + CommitLogSegment.SYNC_MARKER_SIZE;
 -
 -                if (logger.isTraceEnabled())
 -                    logger.trace("Replaying {} between {} and {}", file, reader.getFilePointer(), end);
 -                if (compressor != null)
 -                {
 -                    int uncompressedLength = reader.readInt();
 -                    replayEnd = replayPos + uncompressedLength;
 -                }
 -                else
 -                {
 -                    replayEnd = end;
 -                }
 -
 -                if (segmentId == globalPosition.segment && replayEnd < globalPosition.position)
 -                    // Skip over flushed section.
 -                    continue;
 -
 -                FileDataInput sectionReader = reader;
 -                String errorContext = desc.fileName();
 -                // In the uncompressed case the last non-fully-flushed section can be anywhere in the file.
 -                boolean tolerateErrorsInSection = tolerateTruncation;
 -                if (compressor != null)
 -                {
 -                    // In the compressed case we know if this is the last section.
 -                    tolerateErrorsInSection &= end == reader.length() || end < 0;
 -
 -                    int start = (int) reader.getFilePointer();
 -                    try
 -                    {
 -                        int compressedLength = end - start;
 -                        if (logger.isTraceEnabled())
 -                            logger.trace("Decompressing {} between replay positions {} and {}",
 -                                         file,
 -                                         replayPos,
 -                                         replayEnd);
 -                        if (compressedLength > buffer.length)
 -                            buffer = new byte[(int) (1.2 * compressedLength)];
 -                        reader.readFully(buffer, 0, compressedLength);
 -                        int uncompressedLength = replayEnd - replayPos;
 -                        if (uncompressedLength > uncompressedBuffer.length)
 -                            uncompressedBuffer = new byte[(int) (1.2 * uncompressedLength)];
 -                        compressedLength = compressor.uncompress(buffer, 0, compressedLength, uncompressedBuffer, 0);
 -                        sectionReader = new FileSegmentInputStream(ByteBuffer.wrap(uncompressedBuffer), reader.getPath(), replayPos);
 -                        errorContext = "compressed section at " + start + " in " + errorContext;
 -                    }
 -                    catch (IOException | ArrayIndexOutOfBoundsException e)
 -                    {
 -                        handleReplayError(tolerateErrorsInSection,
 -                                          "Unexpected exception decompressing section at %d: %s",
 -                                          start, e);
 -                        continue;
 -                    }
 -                }
 -
 -                if (!replaySyncSection(sectionReader, replayEnd, desc, errorContext, tolerateErrorsInSection))
 -                    break;
 -            }
 -            logger.debug("Finished reading {}", file);
 -        }
 -    }
 -
 -    public boolean logAndCheckIfShouldSkip(File file, CommitLogDescriptor desc)
 -    {
 -        logger.debug("Replaying {} (CL version {}, messaging version {}, compression {})",
 -                    file.getPath(),
 -                    desc.version,
 -                    desc.getMessagingVersion(),
 -                    desc.compression);
 -
 -        if (globalPosition.segment > desc.id)
 -        {
 -            logger.trace("skipping replay of fully-flushed {}", file);
 -            return true;
 -        }
 -        return false;
 -    }
 -
 -    /**
 -     * Replays a sync section containing a list of mutations.
 -     *
 -     * @return Whether replay should continue with the next section.
 -     */
 -    private boolean replaySyncSection(FileDataInput reader, int end, CommitLogDescriptor desc, String errorContext, boolean tolerateErrors) throws IOException
 -    {
 -         /* read the logs populate Mutation and apply */
 -        while (reader.getFilePointer() < end && !reader.isEOF())
 -        {
 -            long mutationStart = reader.getFilePointer();
 -            if (logger.isTraceEnabled())
 -                logger.trace("Reading mutation at {}", mutationStart);
 -
 -            long claimedCRC32;
 -            int serializedSize;
 -            try
 -            {
 -                // any of the reads may hit EOF
 -                serializedSize = reader.readInt();
 -                if (serializedSize == LEGACY_END_OF_SEGMENT_MARKER)
 -                {
 -                    logger.trace("Encountered end of segment marker at {}", reader.getFilePointer());
 -                    return false;
 -                }
 -
 -                // Mutation must be at LEAST 10 bytes:
 -                // 3 each for a non-empty Keyspace and Key (including the
 -                // 2-byte length from writeUTF/writeWithShortLength) and 4 bytes for column count.
 -                // This prevents CRC by being fooled by special-case garbage in the file; see CASSANDRA-2128
 -                if (serializedSize < 10)
 -                {
 -                    handleReplayError(tolerateErrors,
 -                                      "Invalid mutation size %d at %d in %s",
 -                                      serializedSize, mutationStart, errorContext);
 -                    return false;
 -                }
 -
 -                long claimedSizeChecksum;
 -                if (desc.version < CommitLogDescriptor.VERSION_21)
 -                    claimedSizeChecksum = reader.readLong();
 -                else
 -                    claimedSizeChecksum = reader.readInt() & 0xffffffffL;
 -                checksum.reset();
 -                if (desc.version < CommitLogDescriptor.VERSION_20)
 -                    checksum.update(serializedSize);
 -                else
 -                    updateChecksumInt(checksum, serializedSize);
 -
 -                if (checksum.getValue() != claimedSizeChecksum)
 -                {
 -                    handleReplayError(tolerateErrors,
 -                                      "Mutation size checksum failure at %d in %s",
 -                                      mutationStart, errorContext);
 -                    return false;
 -                }
 -                // ok.
 -
 -                if (serializedSize > buffer.length)
 -                    buffer = new byte[(int) (1.2 * serializedSize)];
 -                reader.readFully(buffer, 0, serializedSize);
 -                if (desc.version < CommitLogDescriptor.VERSION_21)
 -                    claimedCRC32 = reader.readLong();
 -                else
 -                    claimedCRC32 = reader.readInt() & 0xffffffffL;
 -            }
 -            catch (EOFException eof)
 -            {
 -                handleReplayError(tolerateErrors,
 -                                  "Unexpected end of segment",
 -                                  mutationStart, errorContext);
 -                return false; // last CL entry didn't get completely written. that's ok.
 -            }
 -
 -            checksum.update(buffer, 0, serializedSize);
 -            if (claimedCRC32 != checksum.getValue())
 -            {
 -                handleReplayError(tolerateErrors,
 -                                  "Mutation checksum failure at %d in %s",
 -                                  mutationStart, errorContext);
 -                continue;
 -            }
 -            replayMutation(buffer, serializedSize, (int) reader.getFilePointer(), desc);
 -        }
 -        return true;
 -    }
 -
 -    /**
 -     * Deserializes and replays a commit log entry.
 -     */
 -    void replayMutation(byte[] inputBuffer, int size,
 -            final int entryLocation, final CommitLogDescriptor desc) throws IOException
 -    {
 -
 -        final Mutation mutation;
 -        try (RebufferingInputStream bufIn = new DataInputBuffer(inputBuffer, 0, size))
 -        {
 -            mutation = Mutation.serializer.deserialize(bufIn,
 -                                                       desc.getMessagingVersion(),
 -                                                       SerializationHelper.Flag.LOCAL);
 -            // doublecheck that what we read is [still] valid for the current schema
 -            for (PartitionUpdate upd : mutation.getPartitionUpdates())
 -                upd.validate();
 -        }
 -        catch (UnknownColumnFamilyException ex)
 -        {
 -            if (ex.cfId == null)
 -                return;
 -            AtomicInteger i = invalidMutations.get(ex.cfId);
 -            if (i == null)
 -            {
 -                i = new AtomicInteger(1);
 -                invalidMutations.put(ex.cfId, i);
 -            }
 -            else
 -                i.incrementAndGet();
 -            return;
 -        }
 -        catch (Throwable t)
 -        {
 -            JVMStabilityInspector.inspectThrowable(t);
 -            File f = File.createTempFile("mutation", "dat");
 -
 -            try (DataOutputStream out = new DataOutputStream(new FileOutputStream(f)))
 -            {
 -                out.write(inputBuffer, 0, size);
 -            }
 -
 -            // Checksum passed so this error can't be permissible.
 -            handleReplayError(false,
 -                              "Unexpected error deserializing mutation; saved to %s.  " +
 -                              "This may be caused by replaying a mutation against a table with the same name but incompatible schema.  " +
 -                              "Exception follows: %s",
 -                              f.getAbsolutePath(),
 -                              t);
 -            return;
 -        }
 -
 -        if (logger.isTraceEnabled())
 -            logger.trace("replaying mutation for {}.{}: {}", mutation.getKeyspaceName(), mutation.key(), "{" + StringUtils.join(mutation.getPartitionUpdates().iterator(), ", ") + "}");
 -
 -        Runnable runnable = new WrappedRunnable()
 -        {
 -            public void runMayThrow()
 -            {
 -                if (Schema.instance.getKSMetaData(mutation.getKeyspaceName()) == null)
 -                    return;
 -                if (pointInTimeExceeded(mutation))
 -                    return;
 -
 -                final Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
 -
 -                // Rebuild the mutation, omitting column families that
 -                //    a) the user has requested that we ignore,
 -                //    b) have already been flushed,
 -                // or c) are part of a cf that was dropped.
 -                // Keep in mind that the cf.name() is suspect. do every thing based on the cfid instead.
 -                Mutation newMutation = null;
 -                for (PartitionUpdate update : replayFilter.filter(mutation))
 -                {
 -                    if (Schema.instance.getCF(update.metadata().cfId) == null)
 -                        continue; // dropped
 -
 -                    // replay if current segment is newer than last flushed one or,
 -                    // if it is the last known segment, if we are after the replay position
 -                    if (shouldReplay(update.metadata().cfId, new ReplayPosition(desc.id, entryLocation)))
 -                    {
 -                        if (newMutation == null)
 -                            newMutation = new Mutation(mutation.getKeyspaceName(), mutation.key());
 -                        newMutation.add(update);
 -                        replayedCount.incrementAndGet();
 -                    }
 -                }
 -                if (newMutation != null)
 -                {
 -                    assert !newMutation.isEmpty();
 -
 -                    try
 -                    {
 -                        Uninterruptibles.getUninterruptibly(Keyspace.open(newMutation.getKeyspaceName()).applyFromCommitLog(newMutation));
 -                    }
 -                    catch (ExecutionException e)
 -                    {
 -                        throw Throwables.propagate(e.getCause());
 -                    }
 -
 -                    keyspacesRecovered.add(keyspace);
 -                }
 -            }
 -        };
 -        futures.add(StageManager.getStage(Stage.MUTATION).submit(runnable));
 -        if (futures.size() > MAX_OUTSTANDING_REPLAY_COUNT)
 -        {
 -            FBUtilities.waitOnFutures(futures);
 -            futures.clear();
 -        }
 -    }
 -
      protected boolean pointInTimeExceeded(Mutation fm)
      {
          long restoreTarget = archiver.restorePointInTime;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index a1158be,d2f12bf..e32c204
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@@ -32,13 -40,16 +32,14 @@@ import org.cliffc.high_scale_lib.NonBlo
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
 -import org.apache.cassandra.config.CFMetaData;
 -import org.apache.cassandra.config.DatabaseDescriptor;
 -import org.apache.cassandra.config.Schema;
 +import com.codahale.metrics.Timer;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.db.Mutation;
 +import org.apache.cassandra.db.commitlog.CommitLog.Configuration;
  import org.apache.cassandra.db.partitions.PartitionUpdate;
  import org.apache.cassandra.io.FSWriteError;
 -import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.utils.CLibrary;
+ import org.apache.cassandra.utils.IntegerInterval;
  import org.apache.cassandra.utils.concurrent.OpOrder;
  import org.apache.cassandra.utils.concurrent.WaitQueue;
  
@@@ -461,22 -448,18 +475,19 @@@ public abstract class CommitLogSegmen
       * given context argument is contained in this file, it will only mark the CF as
       * clean if no newer writes have taken place.
       *
--     * @param cfId    the column family ID that is now clean
--     * @param context the optional clean offset
++     * @param cfId           the column family ID that is now clean
++     * @param startPosition  the start of the range that is clean
++     * @param endPosition    the end of the range that is clean
       */
-     public synchronized void markClean(UUID cfId, CommitLogPosition context)
 -    public synchronized void markClean(UUID cfId, ReplayPosition startPosition, ReplayPosition endPosition)
++    public synchronized void markClean(UUID cfId, CommitLogPosition startPosition, CommitLogPosition endPosition)
      {
 -        if (startPosition.segment > id || endPosition.segment < id)
++        if (startPosition.segmentId > id || endPosition.segmentId < id)
+             return;
          if (!cfDirty.containsKey(cfId))
              return;
-         if (context.segmentId == id)
-             markClean(cfId, context.position);
-         else if (context.segmentId > id)
-             markClean(cfId, Integer.MAX_VALUE);
-     }
- 
-     private void markClean(UUID cfId, int position)
-     {
-         ensureAtleast(cfClean, cfId, position);
 -        int start = startPosition.segment == id ? startPosition.position : 0;
 -        int end = endPosition.segment == id ? endPosition.position : Integer.MAX_VALUE;
++        int start = startPosition.segmentId == id ? startPosition.position : 0;
++        int end = endPosition.segmentId == id ? endPosition.position : Integer.MAX_VALUE;
+         cfClean.computeIfAbsent(cfId, k -> new IntegerInterval.Set()).add(start, end);
          removeCleanFromDirty();
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b102173/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------


[03/23] cassandra git commit: Disable passing control to post-flush after flush failure to prevent data loss.

Posted by sl...@apache.org.
Disable passing control to post-flush after flush failure to prevent
data loss.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828

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

Branch: refs/heads/cassandra-3.0
Commit: bd6654733dded3513c2c7acf96df2c364b0c043e
Parents: bc0d1da
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 3 11:32:48 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:35:25 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  45 ++++--
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  87 ++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 311 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8ecc787..1275631 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/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 b64d5de..6e82745 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -99,6 +99,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                               new NamedThreadFactory("MemtablePostFlush"),
                                                                                               "internal");
 
+    // If a flush fails with an error the post-flush is never allowed to continue. This stores the error that caused it
+    // to be able to show an error on following flushes instead of blindly continuing.
+    private static volatile FSWriteError previousFlushFailure = null;
+
     private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
                                                                                             StageManager.KEEPALIVE,
                                                                                             TimeUnit.SECONDS,
@@ -869,12 +873,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         synchronized (data)
         {
+            if (previousFlushFailure != null)
+                throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                              + "no flushes can be carried out until the node is restarted.",
+                                                previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            flushExecutor.execute(flush);
+            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            flushExecutor.submit(flushTask);
             ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
             postFlushExecutor.submit(task);
-            return task;
+
+            @SuppressWarnings("unchecked")
+            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            return future;
         }
     }
 
@@ -967,7 +979,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         final ReplayPosition lastReplayPosition;
-        volatile FSWriteError flushFailure = null;
 
         private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
         {
@@ -1010,16 +1021,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             // must check lastReplayPosition != null because Flush may find that all memtables are clean
             // and so not set a lastReplayPosition
-            // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (lastReplayPosition != null && flushFailure == null)
+            if (lastReplayPosition != null)
             {
                 CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
             }
 
             metric.pendingFlushes.dec();
-
-            if (flushFailure != null)
-                throw flushFailure;
         }
     }
 
@@ -1127,16 +1134,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
                     reclaim(memtable);
                 }
+
+                // signal the post-flush we've done our work
+                // Note: This should not be done in case of error. Read more below.
+                postFlush.latch.countDown();
             }
             catch (FSWriteError e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
-                postFlush.flushFailure = e;
+                // The call above may kill the process or the transports, or ignore the error.
+                // In any case we should not be passing on control to post-flush as a subsequent succeeding flush
+                // could mask the error and:
+                //   - let the commit log discard unpersisted data, resulting in data loss
+                //   - let truncations proceed, with the possibility of resurrecting the unflushed data
+                //   - let snapshots succeed with incomplete data
+
+                // Not passing control on means that all flushes from the moment of failure cannot complete
+                // (including snapshots).
+                // If the disk failure policy is ignore, this will cause memtables and the commit log to grow
+                // unboundedly until the node eventually fails.
+                previousFlushFailure = e;
+                throw e;
             }
-
-            // signal the post-flush we've done our work
-            postFlush.latch.countDown();
         }
 
         private void reclaim(final Memtable memtable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
new file mode 100644
index 0000000..c0023dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOError;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories.DataDirectory;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.CommitLogSegment;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSWriteError;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+ */
+public class OutOfSpaceBase extends CQLTester
+{
+    public void makeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+    }
+
+    public void markDirectoriesUnwriteable()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+    }
+
+    public void flushAndExpectError() throws InterruptedException, ExecutionException
+    {
+        try
+        {
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+            fail("FSWriteError expected.");
+        }
+        catch (ExecutionException e)
+        {
+            // Correct path.
+            Assert.assertTrue(e.getCause() instanceof FSWriteError);
+        }
+
+        // Make sure commit log wasn't discarded.
+        UUID cfid = currentTableMetadata().cfId;
+        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+            if (segment.getDirtyCFIDs().contains(cfid))
+                return;
+        fail("Expected commit log to remain dirty for the affected table.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
new file mode 100644
index 0000000..46d71e4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceDieTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableDie() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
+            flushAndExpectError();
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
new file mode 100644
index 0000000..854de80
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceIgnoreTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableIgnore() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+            flushAndExpectError();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
new file mode 100644
index 0000000..b48df56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceStopTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableStop() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
+            flushAndExpectError();
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
deleted file mode 100644
index 8304aff..0000000
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ /dev/null
@@ -1,157 +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.cql3;
-
-import static junit.framework.Assert.fail;
-
-import java.io.IOError;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config.DiskFailurePolicy;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class OutOfSpaceTest extends CQLTester
-{
-    @Test
-    public void testFlushUnwriteableDie() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
-            flushAndExpectError();
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableStop() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
-            flushAndExpectError();
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableIgnore() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
-            flushAndExpectError();
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-
-        // Next flush should succeed.
-        makeTable();
-        flush();
-    }
-
-    public void makeTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-    }
-
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.directories.getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
-    public void flushAndExpectError() throws InterruptedException, ExecutionException
-    {
-        try
-        {
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
-            fail("FSWriteError expected.");
-        }
-        catch (ExecutionException e)
-        {
-            // Correct path.
-            Assert.assertTrue(e.getCause() instanceof FSWriteError);
-        }
-
-        // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
-        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
-                return;
-        fail("Expected commit log to remain dirty for the affected table.");
-    }
-}


[13/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/utils/IntegerInterval.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IntegerInterval.java b/src/java/org/apache/cassandra/utils/IntegerInterval.java
new file mode 100644
index 0000000..03ad6e0
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/IntegerInterval.java
@@ -0,0 +1,227 @@
+package org.apache.cassandra.utils;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+/**
+ * Mutable integer interval class, thread-safe.
+ * Represents the interval [lower,upper].
+ */
+public class IntegerInterval
+{
+    volatile long interval;
+    private static AtomicLongFieldUpdater<IntegerInterval> intervalUpdater =
+            AtomicLongFieldUpdater.newUpdater(IntegerInterval.class, "interval");
+
+    private IntegerInterval(long interval)
+    {
+        this.interval = interval;
+    }
+
+    public IntegerInterval(int lower, int upper)
+    {
+        this(make(lower, upper));
+    }
+
+    public IntegerInterval(IntegerInterval src)
+    {
+        this(src.interval);
+    }
+
+    public int lower()
+    {
+        return lower(interval);
+    }
+
+    public int upper()
+    {
+        return upper(interval);
+    }
+
+    /**
+     * Expands the interval to cover the given value by extending one of its sides if necessary.
+     * Mutates this. Thread-safe.
+     */
+    public void expandToCover(int value)
+    {
+        long prev;
+        int lower;
+        int upper;
+        do
+        {
+            prev = interval;
+            upper = upper(prev);
+            lower = lower(prev);
+            if (value > upper) // common case
+                upper = value;
+            else if (value < lower)
+                lower = value;
+        }
+        while (!intervalUpdater.compareAndSet(this, prev, make(lower, upper)));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Long.hashCode(interval);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (getClass() != obj.getClass())
+            return false;
+        IntegerInterval other = (IntegerInterval) obj;
+        return interval == other.interval;
+    }
+
+    public String toString()
+    {
+        long interval = this.interval;
+        return "[" + lower(interval) + "," + upper(interval) + "]";
+    }
+
+    private static long make(int lower, int upper)
+    {
+        assert lower <= upper;
+        return ((lower & 0xFFFFFFFFL) << 32) | upper & 0xFFFFFFFFL;
+    }
+
+    private static int lower(long interval)
+    {
+        return (int) (interval >>> 32);
+    }
+
+    private static int upper(long interval)
+    {
+        return (int) interval;
+    }
+
+
+    /**
+     * A mutable set of closed integer intervals, stored in normalized form (i.e. where overlapping intervals are
+     * converted to a single interval covering both). Thread-safe.
+     */
+    public static class Set
+    {
+        static long[] EMPTY = new long[0];
+
+        private volatile long[] ranges = EMPTY;
+
+        /**
+         * Adds an interval to the set, performing the necessary normalization.
+         */
+        public synchronized void add(int start, int end)
+        {
+            assert start <= end;
+            long[] ranges, newRanges;
+            {
+                ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+
+                // extend ourselves to cover any ranges we overlap
+                // record directly preceding our end may extend past us, so take the max of our end and its
+                int rpos = Arrays.binarySearch(ranges, ((end & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL); // floor (i.e. greatest <=) of the end position
+                if (rpos < 0)
+                    rpos = (-1 - rpos) - 1;
+                if (rpos >= 0)
+                {
+                    int extend = upper(ranges[rpos]);
+                    if (extend > end)
+                        end = extend;
+                }
+    
+                // record directly preceding our start may extend into us; if it does, we take it as our start
+                int lpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0); // lower (i.e. greatest <) of the start position
+                if (lpos < 0)
+                    lpos = -1 - lpos;
+                lpos -= 1;
+                if (lpos >= 0)
+                {
+                    if (upper(ranges[lpos]) >= start)
+                    {
+                        start = lower(ranges[lpos]);
+                        --lpos;
+                    }
+                }
+    
+                newRanges = new long[ranges.length - (rpos - lpos) + 1];
+                int dest = 0;
+                for (int i = 0; i <= lpos; ++i)
+                    newRanges[dest++] = ranges[i];
+                newRanges[dest++] = make(start, end);
+                for (int i = rpos + 1; i < ranges.length; ++i)
+                    newRanges[dest++] = ranges[i];
+            }
+            this.ranges = newRanges;
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(IntegerInterval iv)
+        {
+            long l = iv.interval;
+            return covers(lower(l), upper(l));
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(int start, int end)
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            int rpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL);        // floor (i.e. greatest <=) of the end position
+            if (rpos < 0)
+                rpos = (-1 - rpos) - 1;
+            if (rpos == -1)
+                return false;
+            return upper(ranges[rpos]) >= end;
+        }
+
+        /**
+         * Returns a lower bound for the whole set. Will throw if set is not empty.
+         */
+        public int lowerBound()
+        {
+            return lower(ranges[0]);
+        }
+
+        /**
+         * Returns an upper bound for the whole set. Will throw if set is not empty.
+         */
+        public int upperBound()
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            return upper(ranges[ranges.length - 1]);
+        }
+
+        public Collection<IntegerInterval> intervals()
+        {
+            return Lists.transform(Longs.asList(ranges), iv -> new IntegerInterval(iv));
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Arrays.hashCode(ranges);
+        }
+
+        @Override
+        public boolean equals(Object obj)
+        {
+            if (getClass() != obj.getClass())
+                return false;
+            Set other = (Set) obj;
+            return Arrays.equals(ranges, other.ranges);
+        }
+
+        public String toString()
+        {
+            return "[" + intervals().stream().map(IntegerInterval::toString).collect(Collectors.joining(", ")) + "]";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2df95c9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
new file mode 100644
index 0000000..c90b58d
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..76480b1
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+2048618157
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
new file mode 100644
index 0000000..3c716e9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
new file mode 100644
index 0000000..43beef3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..69c19f3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7027017
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..4b4078b
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+892998706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b2f5171
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..535d7df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..42876b0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8de00de
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..70a8c08
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1609623183
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..690d2e4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..d5bf83b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..5ff1f27
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7c47153
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..a6875fa
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1205036423
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b31055b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..3463560
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0b7faea
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
new file mode 100644
index 0000000..5862341
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..ee0485a
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+34605693
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
new file mode 100644
index 0000000..b3094bf
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
new file mode 100644
index 0000000..124f9a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..4a00428
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..36c7d92
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4017973941
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..ac35208
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0d9c077
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8aadb48
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..7bb4450
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1545836769
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..59e65ca
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c707d9e
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..6a5f57f
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..45b9e94
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4272819930
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c3299a0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index d517055..02b26c7 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -246,7 +246,7 @@ public class CommitLogStressTest
             verifySizes(commitLog);
 
             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
-                                               discardedPos);
+                    ReplayPosition.NONE, discardedPos);
             threads.clear();
             System.out.format("Discarded at %s\n", discardedPos);
             verifySizes(commitLog);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 e7b1ffa..d04ca9b 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -19,7 +19,9 @@ package org.apache.cassandra;
  *
  */
 
+import java.io.Closeable;
 import java.io.EOFException;
+import java.io.IOError;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -39,6 +41,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -57,7 +60,6 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -651,4 +653,21 @@ public class Util
             return queryStorage(cfs, orderGroup);
         }
     }
+
+    public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)
+    {
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+        return () -> BlacklistedDirectories.clearUnwritableUnsafe();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index a213edf..e3dc220 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -385,9 +385,9 @@ public abstract class CQLTester
     {
         try
         {
-            String currentTable = currentTable();
-            if (currentTable != null)
-                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceMajorCompaction();
+            ColumnFamilyStore store = getCurrentColumnFamilyStore();
+            if (store != null)
+                store.forceMajorCompaction();
         }
         catch (InterruptedException | ExecutionException e)
         {
@@ -397,9 +397,9 @@ public abstract class CQLTester
 
     public void cleanupCache()
     {
-        String currentTable = currentTable();
-        if (currentTable != null)
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
+        ColumnFamilyStore store = getCurrentColumnFamilyStore();
+        if (store != null)
+            store.cleanupCache();
     }
 
     public static FunctionName parseFunctionName(String qualifiedName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
index 1527b1e..26e7fe2 100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
@@ -19,18 +19,16 @@ package org.apache.cassandra.cql3;
 
 import static junit.framework.Assert.fail;
 
-import java.io.IOError;
+import java.io.Closeable;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.Keyspace;
@@ -48,12 +46,11 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableDie() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         KillerForTests killerForTests = new KillerForTests();
         JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
             flushAndExpectError();
@@ -71,10 +68,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableStop() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
             flushAndExpectError();
@@ -90,10 +86,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableIgnore() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
             flushAndExpectError();
@@ -104,7 +99,6 @@ public class OutOfSpaceTest extends CQLTester
         }
 
         // Next flush should succeed.
-        makeTable();
         flush();
     }
 
@@ -117,23 +111,6 @@ public class OutOfSpaceTest extends CQLTester
             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
     }
 
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
     public void flushAndExpectError() throws InterruptedException, ExecutionException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 39ba886..9a0ddb8 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -18,12 +18,7 @@
 */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
@@ -31,6 +26,7 @@ import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
@@ -46,18 +42,18 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.compress.DeflateCompressor;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -235,7 +231,7 @@ public class CommitLogTest
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 
         UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -262,7 +258,7 @@ public class CommitLogTest
         // "Flush": this won't delete anything
         UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
         CommitLog.instance.sync(true);
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid1, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
@@ -283,7 +279,7 @@ public class CommitLogTest
         // didn't write anything on cf1 since last flush (and we flush cf2)
 
         UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
@@ -494,8 +490,8 @@ public class CommitLogTest
             ReplayPosition position = CommitLog.instance.getContext();
             for (Keyspace ks : Keyspace.system())
                 for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
-                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
+                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, ReplayPosition.NONE, position);
+            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, ReplayPosition.NONE, position);
             assertEquals(1, CommitLog.instance.activeSegments());
         }
         finally
@@ -532,5 +528,138 @@ public class CommitLogTest
             DatabaseDescriptor.setAutoSnapshot(originalState);
         }
     }
+
+    @Test
+    public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+
+            for (int i = 0 ; i < 5 ; i++)
+            {
+                new RowUpdateBuilder(cfs.metadata, 0, "k")
+                    .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                    .build()
+                    .apply();
+
+                if (i == 2)
+                {
+                    try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                    {
+                        cfs.forceBlockingFlush();
+                    }
+                    catch (Throwable t)
+                    {
+                        // expected. Cause (after some wrappings) should be a write error
+                        while (!(t instanceof FSWriteError))
+                            t = t.getCause();
+                    }
+                }
+                else
+                    cfs.forceBlockingFlush();
+            }
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+        // If retries work subsequent flushes should clear up error and this should change to expect 0.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+            throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        for (int i = 0 ; i < 5 ; i++)
+        {
+            new RowUpdateBuilder(cfs.metadata, 0, "k")
+                .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                .build()
+                .apply();
+
+            Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+            if (i == 2)
+                current.makeUnflushable();
+
+            flushAction.accept(cfs, current);
+        }
+        if (performCompaction)
+            cfs.forceMajorCompaction();
+        // Make sure metadata saves and reads fine
+        for (SSTableReader reader : cfs.getLiveSSTables())
+            reader.reloadSSTableMetadata();
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+        // persisted all data in the commit log. Because we know there was an error, there must be something left to
+        // replay.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+    {
+        try
+        {
+            cfs.forceBlockingFlush();
+        }
+        catch (Throwable t)
+        {
+            // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+            while (!(t instanceof FSWriteError))
+                t = t.getCause();
+            // Wait for started flushes to complete.
+            cfs.switchMemtableIfCurrent(current);
+        }
+    };
+
+    BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+    {
+        // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+        // flushed data.
+        // This does not stop on errors and should retain segments for which flushing failed.
+        CommitLog.instance.forceRecycleAllSegments();
+
+        // Wait for started flushes to complete.
+        cfs.switchMemtableIfCurrent(current);
+    };
+
+    @Test
+    public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, false);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, false);
+    }
+
+    @Test
+    public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, true);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, true);
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
index 96e6ad6..cd1f295 100644
--- a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
@@ -80,9 +80,9 @@ public class NeverPurgeTest extends CQLTester
         execute("DELETE FROM %s WHERE a=3");
         cfs.forceBlockingFlush();
         cfs.enableAutoCompaction();
-        while (cfs.getSSTables().size() > 1)
+        while (cfs.getLiveSSTables().size() > 1)
             Thread.sleep(100);
-        verifyContainsTombstones(cfs.getSSTables(), 3);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 3);
     }
 
     private void testHelper(String deletionStatement) throws Throwable
@@ -94,7 +94,7 @@ public class NeverPurgeTest extends CQLTester
         Thread.sleep(1000);
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
-        verifyContainsTombstones(cfs.getSSTables(), 1);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 1);
     }
 
     private void verifyContainsTombstones(Collection<SSTableReader> sstables, int expectedTombstoneCount) throws Exception

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index b8de711..479e4e2 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -300,9 +300,6 @@ public class TrackerTest
         SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
         tracker.replaceFlushed(prev2, Collections.singleton(reader));
         Assert.assertEquals(1, tracker.getView().sstables.size());
-        Assert.assertEquals(1, tracker.getView().premature.size());
-        tracker.permitCompactionOfFlushed(singleton(reader));
-        Assert.assertEquals(0, tracker.getView().premature.size());
         Assert.assertEquals(1, listener.received.size());
         Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
         listener.received.clear();
@@ -318,13 +315,14 @@ public class TrackerTest
         tracker.markFlushing(prev1);
         reader = MockSchema.sstable(0, 10, true, cfs);
         cfs.invalidate(false);
-        tracker.replaceFlushed(prev1, Collections.singleton(reader));
-        tracker.permitCompactionOfFlushed(Collections.singleton(reader));
+        tracker.replaceFlushed(prev1, singleton(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, (((SSTableDeletingNotification) listener.received.get(0)).deleting));
-        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+        Assert.assertEquals(3, listener.received.size());
+        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
+        Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
+        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
         DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index a5dceca..7e4ced1 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -212,6 +212,6 @@ public class ViewTest
         for (int i = 0 ; i < sstableCount ; i++)
             sstables.add(MockSchema.sstable(i, cfs));
         return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
-                        Collections.<SSTableReader, SSTableReader>emptyMap(), Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+                        Collections.<SSTableReader, SSTableReader>emptyMap(), SSTableIntervalTree.build(sstables));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 62228e3..cefee52 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -71,7 +71,7 @@ public class LegacySSTableTest
      * See {@link #testGenerateSstables()} to generate sstables.
      * Take care on commit as you need to add the sstable files using {@code git add -f}
      */
-    public static final String[] legacyVersions = {"mb", "ma", "la", "ka", "jb"};
+    public static final String[] legacyVersions = {"mc", "mb", "ma", "la", "ka", "jb"};
 
     // 1200 chars
     static final String longString = "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 c842b7f..a7e822f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -781,7 +781,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
 
         cfs.addSSTable(writeFile(cfs, 1000));
 
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final Token firstToken = allSSTables.iterator().next().first.getToken();
         DatabaseDescriptor.setSSTablePreempiveOpenIntervalInMB(1);
@@ -880,7 +880,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
         truncate(cfs);
 
         cfs.addSSTable(writeFile(cfs, 100));
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final AtomicBoolean done = new AtomicBoolean(false);
         final AtomicBoolean failed = new AtomicBoolean(false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 93365ef..de12d57 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.io.sstable.Component;
@@ -85,8 +86,7 @@ public class MetadataSerializerTest
 
         CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                          .commitLogLowerBound(cllb)
-                                          .commitLogUpperBound(club);
+                                          .commitLogIntervals(new IntervalSet(cllb, club));
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
@@ -106,6 +106,18 @@ public class MetadataSerializerTest
         testOldReadsNew("ma", "mb");
     }
 
+    @Test
+    public void testMaReadMc() throws IOException
+    {
+        testOldReadsNew("ma", "mc");
+    }
+
+    @Test
+    public void testMbReadMc() throws IOException
+    {
+        testOldReadsNew("mb", "mc");
+    }
+
     public void testOldReadsNew(String oldV, String newV) throws IOException
     {
         Map<MetadataType, MetadataComponent> originalMetadata = constructMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
new file mode 100644
index 0000000..44843fd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
@@ -0,0 +1,326 @@
+package org.apache.cassandra.utils;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Futures;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.IntegerInterval.Set;
+
+public class IntegerIntervalsTest
+{
+    int[] values = new int[] { Integer.MIN_VALUE, -2, -1, 0, 5, 9, 13, Integer.MAX_VALUE };
+
+    @Test
+    public void testMake()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                assertEquals(values[i], iv.lower());
+                assertEquals(values[j], iv.upper());
+            }
+        }
+
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = 0; j < i; ++j)
+            {
+                try
+                {
+                    iv = new IntegerInterval(values[i], values[j]);
+                    fail("Assertion not thrown: " + values[i] + ", " + values[j]);
+                }
+                catch (AssertionError e)
+                {
+                    // expected
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverSingleThread()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                int k = 0;
+                for (; k < i; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[k], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < j; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < values.length; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[k], v.upper());
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 1000000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval interval = new IntegerInterval(0, 0);
+        int min = 0;
+        int max = 0;
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed).ints(streamSize).forEach(v -> interval.expandToCover(v));
+                return null;
+            });
+            min = Math.min(min, new Random(seed).ints(streamSize).min().getAsInt());
+            max = Math.max(max, new Random(seed).ints(streamSize).max().getAsInt());
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(min, interval.lower());
+        assertEquals(max, interval.upper());
+    }
+
+    void testSetAdd(int l, int r, Integer... expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertArrayEquals(expected, s
+                                    .intervals()
+                                    .stream()
+                                    .flatMap(x -> ImmutableList.of(x.lower(), x.upper()).stream())
+                                    .toArray());
+    }
+
+    void testSetAdd(int l, int r, String expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertEquals(expected, s.toString());
+    }
+
+    @Test
+    public void testSetAdd()
+    {
+        testSetAdd(Integer.MIN_VALUE, -4, Integer.MIN_VALUE, -4, -3, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -3, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -2, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -1, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 0, Integer.MIN_VALUE, 0, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 1, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 2, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 3, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, Integer.MAX_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE);
+
+        testSetAdd(-5, -4, "[[-5,-4], [-3,-1], [1,3]]");
+        testSetAdd(-5, -3, -5, -1, 1, 3);
+        testSetAdd(-5, -2, -5, -1, 1, 3);
+        testSetAdd(-5, -1, -5, -1, 1, 3);
+        testSetAdd(-5, 0, -5, 0, 1, 3);
+        testSetAdd(-5, 1, -5, 3);
+        testSetAdd(-5, 2, -5, 3);
+        testSetAdd(-5, 3, -5, 3);
+        testSetAdd(-5, 4, -5, 4);
+        testSetAdd(-5, Integer.MAX_VALUE, -5, Integer.MAX_VALUE);
+
+        testSetAdd(-3, -3, -3, -1, 1, 3);
+        testSetAdd(-3, -2, -3, -1, 1, 3);
+        testSetAdd(-3, -1, -3, -1, 1, 3);
+        testSetAdd(-3, 0, -3, 0, 1, 3);
+        testSetAdd(-3, 1, "[[-3,3]]");
+        testSetAdd(-3, 2, -3, 3);
+        testSetAdd(-3, 3, -3, 3);
+        testSetAdd(-3, 4, -3, 4);
+        testSetAdd(-3, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-2, -2, -3, -1, 1, 3);
+        testSetAdd(-2, -1, -3, -1, 1, 3);
+        testSetAdd(-2, 0, "[[-3,0], [1,3]]");
+        testSetAdd(-2, 1, -3, 3);
+        testSetAdd(-2, 2, -3, 3);
+        testSetAdd(-2, 3, -3, 3);
+        testSetAdd(-2, 4, -3, 4);
+        testSetAdd(-2, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-1, -1, -3, -1, 1, 3);
+        testSetAdd(-1, 0, -3, 0, 1, 3);
+        testSetAdd(-1, 1, -3, 3);
+        testSetAdd(-1, 2, -3, 3);
+        testSetAdd(-1, 3, -3, 3);
+        testSetAdd(-1, 4, -3, 4);
+        testSetAdd(-1, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(0, 0, -3, -1, 0, 0, 1, 3);
+        testSetAdd(0, 1, -3, -1, 0, 3);
+        testSetAdd(0, 2, -3, -1, 0, 3);
+        testSetAdd(0, 3, -3, -1, 0, 3);
+        testSetAdd(0, 4, -3, -1, 0, 4);
+        testSetAdd(0, Integer.MAX_VALUE, -3, -1, 0, Integer.MAX_VALUE);
+
+        testSetAdd(1, 1, -3, -1, 1, 3);
+        testSetAdd(1, 2, -3, -1, 1, 3);
+        testSetAdd(1, 3, -3, -1, 1, 3);
+        testSetAdd(1, 4, -3, -1, 1, 4);
+        testSetAdd(1, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(2, 2, -3, -1, 1, 3);
+        testSetAdd(2, 3, -3, -1, 1, 3);
+        testSetAdd(2, 4, -3, -1, 1, 4);
+        testSetAdd(2, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(3, 3, "[[-3,-1], [1,3]]");
+        testSetAdd(3, 4, -3, -1, 1, 4);
+        testSetAdd(3, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(4, 5, -3, -1, 1, 3, 4, 5);
+        testSetAdd(4, Integer.MAX_VALUE, -3, -1, 1, 3, 4, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testSetAddMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 10000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval.Set st = new IntegerInterval.Set();
+        final IntegerInterval.Set mt = new IntegerInterval.Set();
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed)
+                    .ints(streamSize)
+                    .forEach(v -> mt.add(v, v + 5));
+                return null;
+            });
+            new Random(seed)
+                .ints(streamSize)
+                .forEach(v -> st.add(v, v + 5));
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(st, mt);
+    }
+
+    void testSetCovers(int l, int r, boolean expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        assertEquals(expected, s.covers(new IntegerInterval(l, r)));
+    }
+
+
+    @Test
+    public void testSetCovers()
+    {
+        testSetCovers(Integer.MIN_VALUE, -4, false);
+        testSetCovers(Integer.MIN_VALUE, -3, false);
+        testSetCovers(Integer.MIN_VALUE, -2, false);
+        testSetCovers(Integer.MIN_VALUE, -1, false);
+        testSetCovers(Integer.MIN_VALUE, 0, false);
+        testSetCovers(Integer.MIN_VALUE, 1, false);
+        testSetCovers(Integer.MIN_VALUE, 2, false);
+        testSetCovers(Integer.MIN_VALUE, 3, false);
+        testSetCovers(Integer.MIN_VALUE, Integer.MAX_VALUE, false);
+
+        testSetCovers(-5, -4, false);
+        testSetCovers(-5, -3, false);
+        testSetCovers(-5, -2, false);
+        testSetCovers(-5, -1, false);
+        testSetCovers(-5, 0, false);
+        testSetCovers(-5, 1, false);
+        testSetCovers(-5, 2, false);
+        testSetCovers(-5, 3, false);
+        testSetCovers(-5, 4, false);
+        testSetCovers(-5, Integer.MAX_VALUE, false);
+
+        testSetCovers(-3, -3, true);
+        testSetCovers(-3, -2, true);
+        testSetCovers(-3, -1, true);
+        testSetCovers(-3, 0, false);
+        testSetCovers(-3, 1, false);
+        testSetCovers(-3, 2, false);
+        testSetCovers(-3, 3, false);
+        testSetCovers(-3, 4, false);
+        testSetCovers(-3, Integer.MAX_VALUE, false);
+
+        testSetCovers(-2, -2, true);
+        testSetCovers(-2, -1, true);
+        testSetCovers(-2, 0, false);
+        testSetCovers(-2, 1, false);
+        testSetCovers(-2, 2, false);
+        testSetCovers(-2, 3, false);
+        testSetCovers(-2, 4, false);
+        testSetCovers(-2, Integer.MAX_VALUE, false);
+
+        testSetCovers(-1, -1, true);
+        testSetCovers(-1, 0, false);
+        testSetCovers(-1, 1, false);
+        testSetCovers(-1, 2, false);
+        testSetCovers(-1, 3, false);
+        testSetCovers(-1, 4, false);
+        testSetCovers(-1, Integer.MAX_VALUE, false);
+
+        testSetCovers(0, 0, false);
+        testSetCovers(0, 1, false);
+        testSetCovers(0, 2, false);
+        testSetCovers(0, 3, false);
+        testSetCovers(0, 4, false);
+        testSetCovers(0, Integer.MAX_VALUE, false);
+
+        testSetCovers(1, 1, true);
+        testSetCovers(1, 2, true);
+        testSetCovers(1, 3, true);
+        testSetCovers(1, 4, false);
+        testSetCovers(1, Integer.MAX_VALUE, false);
+
+        testSetCovers(2, 2, true);
+        testSetCovers(2, 3, true);
+        testSetCovers(2, 4, false);
+        testSetCovers(2, Integer.MAX_VALUE, false);
+
+        testSetCovers(3, 3, true);
+        testSetCovers(3, 4, false);
+        testSetCovers(3, Integer.MAX_VALUE, false);
+
+        testSetCovers(4, 5, false);
+        testSetCovers(4, Integer.MAX_VALUE, false);
+    }
+}


[12/23] cassandra git commit: Merge commit '6dc1745' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '6dc1745' into cassandra-3.0

* commit '6dc1745':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/trunk
Commit: cf85f520c768a6494281dd5e94fb12b0b07dd1b0
Parents: b66e5a1 6dc1745
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:37:43 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:43 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[23/23] cassandra git commit: Merge branch 'cassandra-3.9' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-3.9' into trunk

* cassandra-3.9:
  Change commitlog and sstables to track dirty and clean intervals.
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/trunk
Commit: 624ed7838bafa96c2083d5a10ebe9ef44f12dcf8
Parents: 7fe4309 7b10217
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:43:46 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:48:18 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/BlacklistedDirectories.java    |  13 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  70 +---
 .../org/apache/cassandra/db/Directories.java    |   2 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  21 +-
 .../AbstractCommitLogSegmentManager.java        |   4 +-
 .../cassandra/db/commitlog/CommitLog.java       |  11 +-
 .../db/commitlog/CommitLogReplayer.java         | 105 ++----
 .../db/commitlog/CommitLogSegment.java          |  82 +++--
 .../cassandra/db/commitlog/IntervalSet.java     | 192 +++++++++++
 .../compaction/AbstractCompactionStrategy.java  |   3 +
 .../compaction/CompactionStrategyManager.java   |   3 +
 .../apache/cassandra/db/lifecycle/Tracker.java  |  45 +--
 .../org/apache/cassandra/db/lifecycle/View.java |  35 +-
 .../cassandra/io/sstable/format/Version.java    |   2 +
 .../io/sstable/format/big/BigFormat.java        |  12 +-
 .../metadata/LegacyMetadataSerializer.java      |  17 +-
 .../io/sstable/metadata/MetadataCollector.java  |  38 +--
 .../io/sstable/metadata/StatsMetadata.java      |  44 +--
 .../cassandra/tools/SSTableMetadataViewer.java  |   3 +-
 .../apache/cassandra/utils/IntegerInterval.java | 227 +++++++++++++
 .../legacy_mc_clust/mc-1-big-CompressionInfo.db | Bin 0 -> 83 bytes
 .../legacy_mc_clust/mc-1-big-Data.db            | Bin 0 -> 5355 bytes
 .../legacy_mc_clust/mc-1-big-Digest.crc32       |   1 +
 .../legacy_mc_clust/mc-1-big-Filter.db          | Bin 0 -> 24 bytes
 .../legacy_mc_clust/mc-1-big-Index.db           | Bin 0 -> 157553 bytes
 .../legacy_mc_clust/mc-1-big-Statistics.db      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust/mc-1-big-Summary.db         | Bin 0 -> 47 bytes
 .../legacy_mc_clust/mc-1-big-TOC.txt            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_mc_clust_compact/mc-1-big-Data.db    | Bin 0 -> 5382 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_compact/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_compact/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7086 bytes
 .../legacy_mc_clust_compact/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_compact/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../legacy_mc_clust_counter/mc-1-big-Data.db    | Bin 0 -> 4631 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_clust_counter/mc-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_mc_clust_counter/mc-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../legacy_mc_clust_counter/mc-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_mc_clust_counter/mc-1-big-TOC.txt    |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 4625 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 7095 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple/mc-1-big-Data.db           | Bin 0 -> 89 bytes
 .../legacy_mc_simple/mc-1-big-Digest.crc32      |   1 +
 .../legacy_mc_simple/mc-1-big-Filter.db         | Bin 0 -> 24 bytes
 .../legacy_mc_simple/mc-1-big-Index.db          | Bin 0 -> 26 bytes
 .../legacy_mc_simple/mc-1-big-Statistics.db     | Bin 0 -> 4639 bytes
 .../legacy_mc_simple/mc-1-big-Summary.db        | Bin 0 -> 47 bytes
 .../legacy_mc_simple/mc-1-big-TOC.txt           |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_compact/mc-1-big-Data.db   | Bin 0 -> 91 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_compact/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_compact/mc-1-big-Index.db  | Bin 0 -> 26 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4680 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_compact/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_mc_simple_counter/mc-1-big-Data.db   | Bin 0 -> 110 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../legacy_mc_simple_counter/mc-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_mc_simple_counter/mc-1-big-Index.db  | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4648 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_mc_simple_counter/mc-1-big-TOC.txt   |   8 +
 .../mc-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../mc-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../mc-1-big-Digest.crc32                       |   1 +
 .../mc-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../mc-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../mc-1-big-Statistics.db                      | Bin 0 -> 4689 bytes
 .../mc-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../mc-1-big-TOC.txt                            |   8 +
 .../db/commitlog/CommitLogStressTest.java       |   3 +-
 test/unit/org/apache/cassandra/Util.java        |  20 ++
 .../org/apache/cassandra/cql3/CQLTester.java    |  12 +-
 .../apache/cassandra/cql3/OutOfSpaceTest.java   |  33 +-
 .../cassandra/db/commitlog/CommitLogTest.java   | 151 ++++++++-
 .../cassandra/db/compaction/NeverPurgeTest.java |   6 +-
 .../cassandra/db/lifecycle/TrackerTest.java     |  12 +-
 .../apache/cassandra/db/lifecycle/ViewTest.java |   2 +-
 .../cassandra/io/sstable/LegacySSTableTest.java |   2 +-
 .../io/sstable/SSTableRewriterTest.java         |   4 +-
 .../metadata/MetadataSerializerTest.java        |  16 +-
 .../streaming/StreamTransferTaskTest.java       |   2 +-
 .../cassandra/utils/IntegerIntervalsTest.java   | 326 +++++++++++++++++++
 98 files changed, 1223 insertions(+), 368 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogSegmentManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index eb9759e,e32c204..b6eea94
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@@ -36,8 -38,8 +36,9 @@@ import org.apache.cassandra.db.Mutation
  import org.apache.cassandra.db.commitlog.CommitLog.Configuration;
  import org.apache.cassandra.db.partitions.PartitionUpdate;
  import org.apache.cassandra.io.FSWriteError;
 +import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.utils.CLibrary;
+ import org.apache.cassandra.utils.IntegerInterval;
  import org.apache.cassandra.utils.concurrent.OpOrder;
  import org.apache.cassandra.utils.concurrent.WaitQueue;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
index 2dccf3c,14e391b..6cc33f5
--- a/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/LegacyMetadataSerializer.java
@@@ -21,8 -21,11 +21,9 @@@ import java.io.*
  import java.nio.ByteBuffer;
  import java.util.*;
  
 -import com.google.common.collect.Maps;
 -
  import org.apache.cassandra.db.TypeSizes;
  import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.format.Version;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index eb9abcf,196cfbf..730f9f0
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@@ -24,8 -23,8 +24,6 @@@ import java.util.EnumMap
  import java.util.List;
  import java.util.Map;
  
- import com.google.common.collect.Ordering;
 -import com.google.common.collect.Maps;
--
  import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus;
  import com.clearspring.analytics.stream.cardinality.ICardinality;
  import org.apache.cassandra.db.*;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index b455ad7,6686684..acad0c5
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -112,8 -112,7 +112,7 @@@ public class SSTableMetadataViewe
                      out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
-                     out.printf("Minimum replay position: %s%n", stats.commitLogLowerBound);
-                     out.printf("Maximum replay position: %s%n", stats.commitLogUpperBound);
 -                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
++                    out.printf("Replay positions covered: %s%n", stats.commitLogIntervals);
                      out.printf("totalColumnsSet: %s%n", stats.totalColumnsSet);
                      out.printf("totalRows: %s%n", stats.totalRows);
                      out.println("Estimated tombstone drop times:");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 4bc5f6b,6ab7d46..30dffe5
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@@ -277,8 -278,8 +278,8 @@@ public class CommitLogTes
  
          // "Flush": this won't delete anything
          UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
 -        CommitLog.instance.sync(true);
 +        CommitLog.instance.sync();
-         CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getCurrentPosition());
+         CommitLog.instance.discardCompletedSegments(cfid1, CommitLogPosition.NONE, CommitLog.instance.getCurrentPosition());
  
          assertEquals(1, CommitLog.instance.segmentManager.getActiveSegments().size());
  
@@@ -683,5 -693,137 +684,137 @@@
              }
          }
      }
+ 
+     public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+         try
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+ 
+             for (int i = 0 ; i < 5 ; i++)
+             {
+                 new RowUpdateBuilder(cfs.metadata, 0, "k")
+                     .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                     .build()
+                     .apply();
+ 
+                 if (i == 2)
+                 {
+                     try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                     {
+                         cfs.forceBlockingFlush();
+                     }
+                     catch (Throwable t)
+                     {
+                         // expected. Cause (after some wrappings) should be a write error
+                         while (!(t instanceof FSWriteError))
+                             t = t.getCause();
+                     }
+                 }
+                 else
+                     cfs.forceBlockingFlush();
+             }
+         }
+         finally
+         {
+             DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+         }
+ 
 -        CommitLog.instance.sync(true);
++        CommitLog.instance.sync();
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+         // If retries work subsequent flushes should clear up error and this should change to expect 0.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+             throws ExecutionException, InterruptedException, IOException
+     {
+         CommitLog.instance.resetUnsafe(true);
+ 
+         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+ 
+         for (int i = 0 ; i < 5 ; i++)
+         {
+             new RowUpdateBuilder(cfs.metadata, 0, "k")
+                 .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                 .build()
+                 .apply();
+ 
+             Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+             if (i == 2)
+                 current.makeUnflushable();
+ 
+             flushAction.accept(cfs, current);
+         }
+         if (performCompaction)
+             cfs.forceMajorCompaction();
+         // Make sure metadata saves and reads fine
+         for (SSTableReader reader : cfs.getLiveSSTables())
+             reader.reloadSSTableMetadata();
+ 
 -        CommitLog.instance.sync(true);
++        CommitLog.instance.sync();
+         System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+         // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+         // persisted all data in the commit log. Because we know there was an error, there must be something left to
+         // replay.
+         Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+     }
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+     {
+         try
+         {
+             cfs.forceBlockingFlush();
+         }
+         catch (Throwable t)
+         {
+             // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+             while (!(t instanceof FSWriteError))
+                 t = t.getCause();
+             // Wait for started flushes to complete.
+             cfs.switchMemtableIfCurrent(current);
+         }
+     };
+ 
+     BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+     {
+         // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+         // flushed data.
+         // This does not stop on errors and should retain segments for which flushing failed.
+         CommitLog.instance.forceRecycleAllSegments();
+ 
+         // Wait for started flushes to complete.
+         cfs.switchMemtableIfCurrent(current);
+     };
+ 
+     @Test
+     public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, false);
+     }
+ 
+     @Test
+     public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(flush, true);
+     }
+ 
+     @Test
+     public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+     {
+         testOutOfOrderFlushRecovery(recycleSegments, true);
+     }
  }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 62d0479,4bd4489..eb50c11
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@@ -29,9 -30,10 +29,10 @@@ import org.junit.Test
  
  import org.apache.cassandra.SchemaLoader;
  import org.apache.cassandra.config.CFMetaData;
 -import org.apache.cassandra.db.SerializationHeader;
  import org.apache.cassandra.config.DatabaseDescriptor;
 +import org.apache.cassandra.db.SerializationHeader;
  import org.apache.cassandra.db.commitlog.CommitLogPosition;
+ import org.apache.cassandra.db.commitlog.IntervalSet;
  import org.apache.cassandra.dht.RandomPartitioner;
  import org.apache.cassandra.io.sstable.Component;
  import org.apache.cassandra.io.sstable.Descriptor;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/624ed783/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index fe75da1,fe75da1..04be91a
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@@ -134,7 -134,7 +134,7 @@@ public class StreamTransferTaskTes
  
          // create streaming task that streams those two sstables
          StreamTransferTask task = new StreamTransferTask(session, cfs.metadata.cfId);
--        List<Ref<SSTableReader>> refs = new ArrayList<>(cfs.getSSTables().size());
++        List<Ref<SSTableReader>> refs = new ArrayList<>(cfs.getLiveSSTables().size());
          for (SSTableReader sstable : cfs.getLiveSSTables())
          {
              List<Range<Token>> ranges = new ArrayList<>();


[15/23] cassandra git commit: Change commitlog and sstables to track dirty and clean intervals.

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/src/java/org/apache/cassandra/utils/IntegerInterval.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/IntegerInterval.java b/src/java/org/apache/cassandra/utils/IntegerInterval.java
new file mode 100644
index 0000000..03ad6e0
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/IntegerInterval.java
@@ -0,0 +1,227 @@
+package org.apache.cassandra.utils;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+/**
+ * Mutable integer interval class, thread-safe.
+ * Represents the interval [lower,upper].
+ */
+public class IntegerInterval
+{
+    volatile long interval;
+    private static AtomicLongFieldUpdater<IntegerInterval> intervalUpdater =
+            AtomicLongFieldUpdater.newUpdater(IntegerInterval.class, "interval");
+
+    private IntegerInterval(long interval)
+    {
+        this.interval = interval;
+    }
+
+    public IntegerInterval(int lower, int upper)
+    {
+        this(make(lower, upper));
+    }
+
+    public IntegerInterval(IntegerInterval src)
+    {
+        this(src.interval);
+    }
+
+    public int lower()
+    {
+        return lower(interval);
+    }
+
+    public int upper()
+    {
+        return upper(interval);
+    }
+
+    /**
+     * Expands the interval to cover the given value by extending one of its sides if necessary.
+     * Mutates this. Thread-safe.
+     */
+    public void expandToCover(int value)
+    {
+        long prev;
+        int lower;
+        int upper;
+        do
+        {
+            prev = interval;
+            upper = upper(prev);
+            lower = lower(prev);
+            if (value > upper) // common case
+                upper = value;
+            else if (value < lower)
+                lower = value;
+        }
+        while (!intervalUpdater.compareAndSet(this, prev, make(lower, upper)));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Long.hashCode(interval);
+    }
+
+    @Override
+    public boolean equals(Object obj)
+    {
+        if (getClass() != obj.getClass())
+            return false;
+        IntegerInterval other = (IntegerInterval) obj;
+        return interval == other.interval;
+    }
+
+    public String toString()
+    {
+        long interval = this.interval;
+        return "[" + lower(interval) + "," + upper(interval) + "]";
+    }
+
+    private static long make(int lower, int upper)
+    {
+        assert lower <= upper;
+        return ((lower & 0xFFFFFFFFL) << 32) | upper & 0xFFFFFFFFL;
+    }
+
+    private static int lower(long interval)
+    {
+        return (int) (interval >>> 32);
+    }
+
+    private static int upper(long interval)
+    {
+        return (int) interval;
+    }
+
+
+    /**
+     * A mutable set of closed integer intervals, stored in normalized form (i.e. where overlapping intervals are
+     * converted to a single interval covering both). Thread-safe.
+     */
+    public static class Set
+    {
+        static long[] EMPTY = new long[0];
+
+        private volatile long[] ranges = EMPTY;
+
+        /**
+         * Adds an interval to the set, performing the necessary normalization.
+         */
+        public synchronized void add(int start, int end)
+        {
+            assert start <= end;
+            long[] ranges, newRanges;
+            {
+                ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+
+                // extend ourselves to cover any ranges we overlap
+                // record directly preceding our end may extend past us, so take the max of our end and its
+                int rpos = Arrays.binarySearch(ranges, ((end & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL); // floor (i.e. greatest <=) of the end position
+                if (rpos < 0)
+                    rpos = (-1 - rpos) - 1;
+                if (rpos >= 0)
+                {
+                    int extend = upper(ranges[rpos]);
+                    if (extend > end)
+                        end = extend;
+                }
+    
+                // record directly preceding our start may extend into us; if it does, we take it as our start
+                int lpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0); // lower (i.e. greatest <) of the start position
+                if (lpos < 0)
+                    lpos = -1 - lpos;
+                lpos -= 1;
+                if (lpos >= 0)
+                {
+                    if (upper(ranges[lpos]) >= start)
+                    {
+                        start = lower(ranges[lpos]);
+                        --lpos;
+                    }
+                }
+    
+                newRanges = new long[ranges.length - (rpos - lpos) + 1];
+                int dest = 0;
+                for (int i = 0; i <= lpos; ++i)
+                    newRanges[dest++] = ranges[i];
+                newRanges[dest++] = make(start, end);
+                for (int i = rpos + 1; i < ranges.length; ++i)
+                    newRanges[dest++] = ranges[i];
+            }
+            this.ranges = newRanges;
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(IntegerInterval iv)
+        {
+            long l = iv.interval;
+            return covers(lower(l), upper(l));
+        }
+
+        /**
+         * Returns true if the set completely covers the given interval.
+         */
+        public boolean covers(int start, int end)
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            int rpos = Arrays.binarySearch(ranges, ((start & 0xFFFFFFFFL) << 32) | 0xFFFFFFFFL);        // floor (i.e. greatest <=) of the end position
+            if (rpos < 0)
+                rpos = (-1 - rpos) - 1;
+            if (rpos == -1)
+                return false;
+            return upper(ranges[rpos]) >= end;
+        }
+
+        /**
+         * Returns a lower bound for the whole set. Will throw if set is not empty.
+         */
+        public int lowerBound()
+        {
+            return lower(ranges[0]);
+        }
+
+        /**
+         * Returns an upper bound for the whole set. Will throw if set is not empty.
+         */
+        public int upperBound()
+        {
+            long[] ranges = this.ranges; // take local copy to avoid risk of it changing in the midst of operation
+            return upper(ranges[ranges.length - 1]);
+        }
+
+        public Collection<IntegerInterval> intervals()
+        {
+            return Lists.transform(Longs.asList(ranges), iv -> new IntegerInterval(iv));
+        }
+
+        @Override
+        public int hashCode()
+        {
+            return Arrays.hashCode(ranges);
+        }
+
+        @Override
+        public boolean equals(Object obj)
+        {
+            if (getClass() != obj.getClass())
+                return false;
+            Set other = (Set) obj;
+            return Arrays.equals(ranges, other.ranges);
+        }
+
+        public String toString()
+        {
+            return "[" + intervals().stream().map(IntegerInterval::toString).collect(Collectors.joining(", ")) + "]";
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2df95c9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db
new file mode 100644
index 0000000..c90b58d
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..76480b1
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+2048618157
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db
new file mode 100644
index 0000000..3c716e9
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db
new file mode 100644
index 0000000..43beef3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..69c19f3
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7027017
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..4b4078b
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+892998706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b2f5171
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..535d7df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..42876b0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8de00de
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..70a8c08
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1609623183
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..690d2e4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..d5bf83b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..5ff1f27
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..7c47153
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..a6875fa
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1205036423
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..b31055b
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..3463560
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_clust_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0b7faea
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db
new file mode 100644
index 0000000..5862341
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..ee0485a
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+34605693
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db
new file mode 100644
index 0000000..b3094bf
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db
new file mode 100644
index 0000000..124f9a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..4a00428
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..36c7d92
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4017973941
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..ac35208
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..0d9c077
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db
new file mode 100644
index 0000000..8aadb48
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..7bb4450
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+1545836769
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db
new file mode 100644
index 0000000..59e65ca
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c707d9e
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db
new file mode 100644
index 0000000..6a5f57f
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32 b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
new file mode 100644
index 0000000..45b9e94
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Digest.crc32
@@ -0,0 +1 @@
+4272819930
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db
new file mode 100644
index 0000000..c3299a0
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
new file mode 100644
index 0000000..8ef7cb0
--- /dev/null
+++ b/test/data/legacy-sstables/mc/legacy_tables/legacy_mc_simple_counter_compact/mc-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Summary.db
+Filter.db
+Statistics.db
+Index.db
+Data.db
+CompressionInfo.db
+Digest.crc32

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
index d517055..02b26c7 100644
--- a/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
+++ b/test/long/org/apache/cassandra/db/commitlog/CommitLogStressTest.java
@@ -246,7 +246,7 @@ public class CommitLogStressTest
             verifySizes(commitLog);
 
             commitLog.discardCompletedSegments(Schema.instance.getCFMetaData("Keyspace1", "Standard1").cfId,
-                                               discardedPos);
+                    ReplayPosition.NONE, discardedPos);
             threads.clear();
             System.out.format("Discarded at %s\n", discardedPos);
             verifySizes(commitLog);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 e7b1ffa..d04ca9b 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -19,7 +19,9 @@ package org.apache.cassandra;
  *
  */
 
+import java.io.Closeable;
 import java.io.EOFException;
+import java.io.IOError;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
@@ -39,6 +41,7 @@ import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -57,7 +60,6 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -651,4 +653,21 @@ public class Util
             return queryStorage(cfs, orderGroup);
         }
     }
+
+    public static Closeable markDirectoriesUnwriteable(ColumnFamilyStore cfs)
+    {
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+        return () -> BlacklistedDirectories.clearUnwritableUnsafe();
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index a213edf..e3dc220 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -385,9 +385,9 @@ public abstract class CQLTester
     {
         try
         {
-            String currentTable = currentTable();
-            if (currentTable != null)
-                Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).forceMajorCompaction();
+            ColumnFamilyStore store = getCurrentColumnFamilyStore();
+            if (store != null)
+                store.forceMajorCompaction();
         }
         catch (InterruptedException | ExecutionException e)
         {
@@ -397,9 +397,9 @@ public abstract class CQLTester
 
     public void cleanupCache()
     {
-        String currentTable = currentTable();
-        if (currentTable != null)
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable).cleanupCache();
+        ColumnFamilyStore store = getCurrentColumnFamilyStore();
+        if (store != null)
+            store.cleanupCache();
     }
 
     public static FunctionName parseFunctionName(String qualifiedName)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
index 1527b1e..26e7fe2 100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
@@ -19,18 +19,16 @@ package org.apache.cassandra.cql3;
 
 import static junit.framework.Assert.fail;
 
-import java.io.IOError;
+import java.io.Closeable;
 import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Config.DiskFailurePolicy;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.Keyspace;
@@ -48,12 +46,11 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableDie() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         KillerForTests killerForTests = new KillerForTests();
         JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
             flushAndExpectError();
@@ -71,10 +68,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableStop() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
             flushAndExpectError();
@@ -90,10 +86,9 @@ public class OutOfSpaceTest extends CQLTester
     public void testFlushUnwriteableIgnore() throws Throwable
     {
         makeTable();
-        markDirectoriesUnwriteable();
 
         DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
+        try (Closeable c = Util.markDirectoriesUnwriteable(getCurrentColumnFamilyStore()))
         {
             DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
             flushAndExpectError();
@@ -104,7 +99,6 @@ public class OutOfSpaceTest extends CQLTester
         }
 
         // Next flush should succeed.
-        makeTable();
         flush();
     }
 
@@ -117,23 +111,6 @@ public class OutOfSpaceTest extends CQLTester
             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
     }
 
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.getDirectories().getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.getDirectories().getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
     public void flushAndExpectError() throws InterruptedException, ExecutionException
     {
         try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
index 39ba886..9a0ddb8 100644
--- a/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/commitlog/CommitLogTest.java
@@ -18,12 +18,7 @@
 */
 package org.apache.cassandra.db.commitlog;
 
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
+import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Collection;
@@ -31,6 +26,7 @@ import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
+import java.util.function.BiConsumer;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
@@ -46,18 +42,18 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.ParameterizedClass;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RowUpdateBuilder;
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLogReplayer.CommitLogReplayException;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.compress.DeflateCompressor;
 import org.apache.cassandra.io.compress.LZ4Compressor;
 import org.apache.cassandra.io.compress.SnappyCompressor;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -235,7 +231,7 @@ public class CommitLogTest
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 
         UUID cfid2 = m2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -262,7 +258,7 @@ public class CommitLogTest
         // "Flush": this won't delete anything
         UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
         CommitLog.instance.sync(true);
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid1, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
@@ -283,7 +279,7 @@ public class CommitLogTest
         // didn't write anything on cf1 since last flush (and we flush cf2)
 
         UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
+        CommitLog.instance.discardCompletedSegments(cfid2, ReplayPosition.NONE, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
@@ -494,8 +490,8 @@ public class CommitLogTest
             ReplayPosition position = CommitLog.instance.getContext();
             for (Keyspace ks : Keyspace.system())
                 for (ColumnFamilyStore syscfs : ks.getColumnFamilyStores())
-                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, position);
-            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, position);
+                    CommitLog.instance.discardCompletedSegments(syscfs.metadata.cfId, ReplayPosition.NONE, position);
+            CommitLog.instance.discardCompletedSegments(cfs2.metadata.cfId, ReplayPosition.NONE, position);
             assertEquals(1, CommitLog.instance.activeSegments());
         }
         finally
@@ -532,5 +528,138 @@ public class CommitLogTest
             DatabaseDescriptor.setAutoSnapshot(originalState);
         }
     }
+
+    @Test
+    public void testUnwriteableFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+
+            for (int i = 0 ; i < 5 ; i++)
+            {
+                new RowUpdateBuilder(cfs.metadata, 0, "k")
+                    .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                    .build()
+                    .apply();
+
+                if (i == 2)
+                {
+                    try (Closeable c = Util.markDirectoriesUnwriteable(cfs))
+                    {
+                        cfs.forceBlockingFlush();
+                    }
+                    catch (Throwable t)
+                    {
+                        // expected. Cause (after some wrappings) should be a write error
+                        while (!(t instanceof FSWriteError))
+                            t = t.getCause();
+                    }
+                }
+                else
+                    cfs.forceBlockingFlush();
+            }
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // Currently we don't attempt to re-flush a memtable that failed, thus make sure data is replayed by commitlog.
+        // If retries work subsequent flushes should clear up error and this should change to expect 0.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    public void testOutOfOrderFlushRecovery(BiConsumer<ColumnFamilyStore, Memtable> flushAction, boolean performCompaction)
+            throws ExecutionException, InterruptedException, IOException
+    {
+        CommitLog.instance.resetUnsafe(true);
+
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(STANDARD1);
+
+        for (int i = 0 ; i < 5 ; i++)
+        {
+            new RowUpdateBuilder(cfs.metadata, 0, "k")
+                .clustering("c" + i).add("val", ByteBuffer.allocate(100))
+                .build()
+                .apply();
+
+            Memtable current = cfs.getTracker().getView().getCurrentMemtable();
+            if (i == 2)
+                current.makeUnflushable();
+
+            flushAction.accept(cfs, current);
+        }
+        if (performCompaction)
+            cfs.forceMajorCompaction();
+        // Make sure metadata saves and reads fine
+        for (SSTableReader reader : cfs.getLiveSSTables())
+            reader.reloadSSTableMetadata();
+
+        CommitLog.instance.sync(true);
+        System.setProperty("cassandra.replayList", KEYSPACE1 + "." + STANDARD1);
+        // In the absence of error, this should be 0 because forceBlockingFlush/forceRecycleAllSegments would have
+        // persisted all data in the commit log. Because we know there was an error, there must be something left to
+        // replay.
+        Assert.assertEquals(1, CommitLog.instance.resetUnsafe(false));
+    }
+
+    BiConsumer<ColumnFamilyStore, Memtable> flush = (cfs, current) ->
+    {
+        try
+        {
+            cfs.forceBlockingFlush();
+        }
+        catch (Throwable t)
+        {
+            // expected after makeUnflushable. Cause (after some wrappings) should be a write error
+            while (!(t instanceof FSWriteError))
+                t = t.getCause();
+            // Wait for started flushes to complete.
+            cfs.switchMemtableIfCurrent(current);
+        }
+    };
+
+    BiConsumer<ColumnFamilyStore, Memtable> recycleSegments = (cfs, current) ->
+    {
+        // Move to new commit log segment and try to flush all data. Also delete segments that no longer contain
+        // flushed data.
+        // This does not stop on errors and should retain segments for which flushing failed.
+        CommitLog.instance.forceRecycleAllSegments();
+
+        // Wait for started flushes to complete.
+        cfs.switchMemtableIfCurrent(current);
+    };
+
+    @Test
+    public void testOutOfOrderFlushRecovery() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, false);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscard() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, false);
+    }
+
+    @Test
+    public void testOutOfOrderFlushRecoveryWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(flush, true);
+    }
+
+    @Test
+    public void testOutOfOrderLogDiscardWithCompaction() throws ExecutionException, InterruptedException, IOException
+    {
+        testOutOfOrderFlushRecovery(recycleSegments, true);
+    }
 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
index 96e6ad6..cd1f295 100644
--- a/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/NeverPurgeTest.java
@@ -80,9 +80,9 @@ public class NeverPurgeTest extends CQLTester
         execute("DELETE FROM %s WHERE a=3");
         cfs.forceBlockingFlush();
         cfs.enableAutoCompaction();
-        while (cfs.getSSTables().size() > 1)
+        while (cfs.getLiveSSTables().size() > 1)
             Thread.sleep(100);
-        verifyContainsTombstones(cfs.getSSTables(), 3);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 3);
     }
 
     private void testHelper(String deletionStatement) throws Throwable
@@ -94,7 +94,7 @@ public class NeverPurgeTest extends CQLTester
         Thread.sleep(1000);
         cfs.forceBlockingFlush();
         cfs.forceMajorCompaction();
-        verifyContainsTombstones(cfs.getSSTables(), 1);
+        verifyContainsTombstones(cfs.getLiveSSTables(), 1);
     }
 
     private void verifyContainsTombstones(Collection<SSTableReader> sstables, int expectedTombstoneCount) throws Exception

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index b8de711..479e4e2 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/TrackerTest.java
@@ -300,9 +300,6 @@ public class TrackerTest
         SSTableReader reader = MockSchema.sstable(0, 10, false, cfs);
         tracker.replaceFlushed(prev2, Collections.singleton(reader));
         Assert.assertEquals(1, tracker.getView().sstables.size());
-        Assert.assertEquals(1, tracker.getView().premature.size());
-        tracker.permitCompactionOfFlushed(singleton(reader));
-        Assert.assertEquals(0, tracker.getView().premature.size());
         Assert.assertEquals(1, listener.received.size());
         Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
         listener.received.clear();
@@ -318,13 +315,14 @@ public class TrackerTest
         tracker.markFlushing(prev1);
         reader = MockSchema.sstable(0, 10, true, cfs);
         cfs.invalidate(false);
-        tracker.replaceFlushed(prev1, Collections.singleton(reader));
-        tracker.permitCompactionOfFlushed(Collections.singleton(reader));
+        tracker.replaceFlushed(prev1, singleton(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, (((SSTableDeletingNotification) listener.received.get(0)).deleting));
-        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(1)).removed.size());
+        Assert.assertEquals(3, listener.received.size());
+        Assert.assertEquals(singleton(reader), ((SSTableAddedNotification) listener.received.get(0)).added);
+        Assert.assertTrue(listener.received.get(1) instanceof SSTableDeletingNotification);
+        Assert.assertEquals(1, ((SSTableListChangedNotification) listener.received.get(2)).removed.size());
         DatabaseDescriptor.setIncrementalBackupsEnabled(backups);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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
index a5dceca..7e4ced1 100644
--- a/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
+++ b/test/unit/org/apache/cassandra/db/lifecycle/ViewTest.java
@@ -212,6 +212,6 @@ public class ViewTest
         for (int i = 0 ; i < sstableCount ; i++)
             sstables.add(MockSchema.sstable(i, cfs));
         return new View(ImmutableList.copyOf(memtables), Collections.<Memtable>emptyList(), Helpers.identityMap(sstables),
-                        Collections.<SSTableReader, SSTableReader>emptyMap(), Collections.<SSTableReader>emptySet(), SSTableIntervalTree.build(sstables));
+                        Collections.<SSTableReader, SSTableReader>emptyMap(), SSTableIntervalTree.build(sstables));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 62228e3..cefee52 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -71,7 +71,7 @@ public class LegacySSTableTest
      * See {@link #testGenerateSstables()} to generate sstables.
      * Take care on commit as you need to add the sstable files using {@code git add -f}
      */
-    public static final String[] legacyVersions = {"mb", "ma", "la", "ka", "jb"};
+    public static final String[] legacyVersions = {"mc", "mb", "ma", "la", "ka", "jb"};
 
     // 1200 chars
     static final String longString = "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789" +

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/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 c842b7f..a7e822f 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -781,7 +781,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
 
         cfs.addSSTable(writeFile(cfs, 1000));
 
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final Token firstToken = allSSTables.iterator().next().first.getToken();
         DatabaseDescriptor.setSSTablePreempiveOpenIntervalInMB(1);
@@ -880,7 +880,7 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
         truncate(cfs);
 
         cfs.addSSTable(writeFile(cfs, 100));
-        Collection<SSTableReader> allSSTables = cfs.getSSTables();
+        Collection<SSTableReader> allSSTables = cfs.getLiveSSTables();
         assertEquals(1, allSSTables.size());
         final AtomicBoolean done = new AtomicBoolean(false);
         final AtomicBoolean failed = new AtomicBoolean(false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 93365ef..de12d57 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.SerializationHeader;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.commitlog.IntervalSet;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.io.sstable.Component;
@@ -85,8 +86,7 @@ public class MetadataSerializerTest
 
         CFMetaData cfm = SchemaLoader.standardCFMD("ks1", "cf1");
         MetadataCollector collector = new MetadataCollector(cfm.comparator)
-                                          .commitLogLowerBound(cllb)
-                                          .commitLogUpperBound(club);
+                                          .commitLogIntervals(new IntervalSet(cllb, club));
 
         String partitioner = RandomPartitioner.class.getCanonicalName();
         double bfFpChance = 0.1;
@@ -106,6 +106,18 @@ public class MetadataSerializerTest
         testOldReadsNew("ma", "mb");
     }
 
+    @Test
+    public void testMaReadMc() throws IOException
+    {
+        testOldReadsNew("ma", "mc");
+    }
+
+    @Test
+    public void testMbReadMc() throws IOException
+    {
+        testOldReadsNew("mb", "mc");
+    }
+
     public void testOldReadsNew(String oldV, String newV) throws IOException
     {
         Map<MetadataType, MetadataComponent> originalMetadata = constructMetadata();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/904cb5d1/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
new file mode 100644
index 0000000..44843fd
--- /dev/null
+++ b/test/unit/org/apache/cassandra/utils/IntegerIntervalsTest.java
@@ -0,0 +1,326 @@
+package org.apache.cassandra.utils;
+
+import static org.junit.Assert.*;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.Futures;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.IntegerInterval.Set;
+
+public class IntegerIntervalsTest
+{
+    int[] values = new int[] { Integer.MIN_VALUE, -2, -1, 0, 5, 9, 13, Integer.MAX_VALUE };
+
+    @Test
+    public void testMake()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                assertEquals(values[i], iv.lower());
+                assertEquals(values[j], iv.upper());
+            }
+        }
+
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = 0; j < i; ++j)
+            {
+                try
+                {
+                    iv = new IntegerInterval(values[i], values[j]);
+                    fail("Assertion not thrown: " + values[i] + ", " + values[j]);
+                }
+                catch (AssertionError e)
+                {
+                    // expected
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverSingleThread()
+    {
+        IntegerInterval iv;
+        for (int i = 0; i < values.length; ++i)
+        {
+            for (int j = i; j < values.length; ++j)
+            {
+                iv = new IntegerInterval(values[i], values[j]);
+                int k = 0;
+                for (; k < i; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[k], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < j; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[j], v.upper());
+                }
+                for (; k < values.length; ++k)
+                {
+                    IntegerInterval v = new IntegerInterval(iv);
+                    v.expandToCover(values[k]);
+                    assertEquals(values[i], v.lower());
+                    assertEquals(values[k], v.upper());
+                }
+            }
+        }
+    }
+
+    @Test
+    public void testExpandToCoverMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 1000000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval interval = new IntegerInterval(0, 0);
+        int min = 0;
+        int max = 0;
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed).ints(streamSize).forEach(v -> interval.expandToCover(v));
+                return null;
+            });
+            min = Math.min(min, new Random(seed).ints(streamSize).min().getAsInt());
+            max = Math.max(max, new Random(seed).ints(streamSize).max().getAsInt());
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(min, interval.lower());
+        assertEquals(max, interval.upper());
+    }
+
+    void testSetAdd(int l, int r, Integer... expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertArrayEquals(expected, s
+                                    .intervals()
+                                    .stream()
+                                    .flatMap(x -> ImmutableList.of(x.lower(), x.upper()).stream())
+                                    .toArray());
+    }
+
+    void testSetAdd(int l, int r, String expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        s.add(l, r);
+        assertEquals(expected, s.toString());
+    }
+
+    @Test
+    public void testSetAdd()
+    {
+        testSetAdd(Integer.MIN_VALUE, -4, Integer.MIN_VALUE, -4, -3, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -3, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -2, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, -1, Integer.MIN_VALUE, -1, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 0, Integer.MIN_VALUE, 0, 1, 3);
+        testSetAdd(Integer.MIN_VALUE, 1, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 2, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, 3, Integer.MIN_VALUE, 3);
+        testSetAdd(Integer.MIN_VALUE, Integer.MAX_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE);
+
+        testSetAdd(-5, -4, "[[-5,-4], [-3,-1], [1,3]]");
+        testSetAdd(-5, -3, -5, -1, 1, 3);
+        testSetAdd(-5, -2, -5, -1, 1, 3);
+        testSetAdd(-5, -1, -5, -1, 1, 3);
+        testSetAdd(-5, 0, -5, 0, 1, 3);
+        testSetAdd(-5, 1, -5, 3);
+        testSetAdd(-5, 2, -5, 3);
+        testSetAdd(-5, 3, -5, 3);
+        testSetAdd(-5, 4, -5, 4);
+        testSetAdd(-5, Integer.MAX_VALUE, -5, Integer.MAX_VALUE);
+
+        testSetAdd(-3, -3, -3, -1, 1, 3);
+        testSetAdd(-3, -2, -3, -1, 1, 3);
+        testSetAdd(-3, -1, -3, -1, 1, 3);
+        testSetAdd(-3, 0, -3, 0, 1, 3);
+        testSetAdd(-3, 1, "[[-3,3]]");
+        testSetAdd(-3, 2, -3, 3);
+        testSetAdd(-3, 3, -3, 3);
+        testSetAdd(-3, 4, -3, 4);
+        testSetAdd(-3, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-2, -2, -3, -1, 1, 3);
+        testSetAdd(-2, -1, -3, -1, 1, 3);
+        testSetAdd(-2, 0, "[[-3,0], [1,3]]");
+        testSetAdd(-2, 1, -3, 3);
+        testSetAdd(-2, 2, -3, 3);
+        testSetAdd(-2, 3, -3, 3);
+        testSetAdd(-2, 4, -3, 4);
+        testSetAdd(-2, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(-1, -1, -3, -1, 1, 3);
+        testSetAdd(-1, 0, -3, 0, 1, 3);
+        testSetAdd(-1, 1, -3, 3);
+        testSetAdd(-1, 2, -3, 3);
+        testSetAdd(-1, 3, -3, 3);
+        testSetAdd(-1, 4, -3, 4);
+        testSetAdd(-1, Integer.MAX_VALUE, -3, Integer.MAX_VALUE);
+
+        testSetAdd(0, 0, -3, -1, 0, 0, 1, 3);
+        testSetAdd(0, 1, -3, -1, 0, 3);
+        testSetAdd(0, 2, -3, -1, 0, 3);
+        testSetAdd(0, 3, -3, -1, 0, 3);
+        testSetAdd(0, 4, -3, -1, 0, 4);
+        testSetAdd(0, Integer.MAX_VALUE, -3, -1, 0, Integer.MAX_VALUE);
+
+        testSetAdd(1, 1, -3, -1, 1, 3);
+        testSetAdd(1, 2, -3, -1, 1, 3);
+        testSetAdd(1, 3, -3, -1, 1, 3);
+        testSetAdd(1, 4, -3, -1, 1, 4);
+        testSetAdd(1, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(2, 2, -3, -1, 1, 3);
+        testSetAdd(2, 3, -3, -1, 1, 3);
+        testSetAdd(2, 4, -3, -1, 1, 4);
+        testSetAdd(2, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(3, 3, "[[-3,-1], [1,3]]");
+        testSetAdd(3, 4, -3, -1, 1, 4);
+        testSetAdd(3, Integer.MAX_VALUE, -3, -1, 1, Integer.MAX_VALUE);
+
+        testSetAdd(4, 5, -3, -1, 1, 3, 4, 5);
+        testSetAdd(4, Integer.MAX_VALUE, -3, -1, 1, 3, 4, Integer.MAX_VALUE);
+    }
+
+    @Test
+    public void testSetAddMultiThread() throws InterruptedException
+    {
+        Random r = new Random();
+        int threads = 16;
+        int streamSize = 10000;
+        List<Callable<Void>> tasks = new ArrayList<>(threads);
+        final IntegerInterval.Set st = new IntegerInterval.Set();
+        final IntegerInterval.Set mt = new IntegerInterval.Set();
+        for (int i = 0; i < threads; ++i)
+        {
+            final int seed = r.nextInt();
+            tasks.add(() -> {
+                new Random(seed)
+                    .ints(streamSize)
+                    .forEach(v -> mt.add(v, v + 5));
+                return null;
+            });
+            new Random(seed)
+                .ints(streamSize)
+                .forEach(v -> st.add(v, v + 5));
+        }
+        for (Future<?> f : Executors.newFixedThreadPool(threads).invokeAll(tasks))
+            Futures.getUnchecked(f);
+        assertEquals(st, mt);
+    }
+
+    void testSetCovers(int l, int r, boolean expected)
+    {
+        Set s = new Set();
+        s.add(-3, -1);
+        s.add(1, 3);
+        assertEquals(expected, s.covers(new IntegerInterval(l, r)));
+    }
+
+
+    @Test
+    public void testSetCovers()
+    {
+        testSetCovers(Integer.MIN_VALUE, -4, false);
+        testSetCovers(Integer.MIN_VALUE, -3, false);
+        testSetCovers(Integer.MIN_VALUE, -2, false);
+        testSetCovers(Integer.MIN_VALUE, -1, false);
+        testSetCovers(Integer.MIN_VALUE, 0, false);
+        testSetCovers(Integer.MIN_VALUE, 1, false);
+        testSetCovers(Integer.MIN_VALUE, 2, false);
+        testSetCovers(Integer.MIN_VALUE, 3, false);
+        testSetCovers(Integer.MIN_VALUE, Integer.MAX_VALUE, false);
+
+        testSetCovers(-5, -4, false);
+        testSetCovers(-5, -3, false);
+        testSetCovers(-5, -2, false);
+        testSetCovers(-5, -1, false);
+        testSetCovers(-5, 0, false);
+        testSetCovers(-5, 1, false);
+        testSetCovers(-5, 2, false);
+        testSetCovers(-5, 3, false);
+        testSetCovers(-5, 4, false);
+        testSetCovers(-5, Integer.MAX_VALUE, false);
+
+        testSetCovers(-3, -3, true);
+        testSetCovers(-3, -2, true);
+        testSetCovers(-3, -1, true);
+        testSetCovers(-3, 0, false);
+        testSetCovers(-3, 1, false);
+        testSetCovers(-3, 2, false);
+        testSetCovers(-3, 3, false);
+        testSetCovers(-3, 4, false);
+        testSetCovers(-3, Integer.MAX_VALUE, false);
+
+        testSetCovers(-2, -2, true);
+        testSetCovers(-2, -1, true);
+        testSetCovers(-2, 0, false);
+        testSetCovers(-2, 1, false);
+        testSetCovers(-2, 2, false);
+        testSetCovers(-2, 3, false);
+        testSetCovers(-2, 4, false);
+        testSetCovers(-2, Integer.MAX_VALUE, false);
+
+        testSetCovers(-1, -1, true);
+        testSetCovers(-1, 0, false);
+        testSetCovers(-1, 1, false);
+        testSetCovers(-1, 2, false);
+        testSetCovers(-1, 3, false);
+        testSetCovers(-1, 4, false);
+        testSetCovers(-1, Integer.MAX_VALUE, false);
+
+        testSetCovers(0, 0, false);
+        testSetCovers(0, 1, false);
+        testSetCovers(0, 2, false);
+        testSetCovers(0, 3, false);
+        testSetCovers(0, 4, false);
+        testSetCovers(0, Integer.MAX_VALUE, false);
+
+        testSetCovers(1, 1, true);
+        testSetCovers(1, 2, true);
+        testSetCovers(1, 3, true);
+        testSetCovers(1, 4, false);
+        testSetCovers(1, Integer.MAX_VALUE, false);
+
+        testSetCovers(2, 2, true);
+        testSetCovers(2, 3, true);
+        testSetCovers(2, 4, false);
+        testSetCovers(2, Integer.MAX_VALUE, false);
+
+        testSetCovers(3, 3, true);
+        testSetCovers(3, 4, false);
+        testSetCovers(3, Integer.MAX_VALUE, false);
+
+        testSetCovers(4, 5, false);
+        testSetCovers(4, Integer.MAX_VALUE, false);
+    }
+}


[06/23] cassandra git commit: Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

* commit 'bd6654733dded3513c2c7acf96df2c364b0c043e':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-3.0
Commit: 6dc1745edd8d3861d853ee56f49ac67633a753b0
Parents: 0398521 bd66547
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:36:29 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:11 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  67 +++++---
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  95 +++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 336 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 87228d3,1275631..7fcf373
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,13 +1,58 @@@
 +2.2.8
 + * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
 + * Revert CASSANDRA-11427 (CASSANDRA-12351)
 + * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
 + * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
 + * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 + * Synchronize ThriftServer::stop() (CASSANDRA-12105)
 + * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 + * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
 + * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 + * Don't write shadowed range tombstone (CASSANDRA-12030)
 +Merged from 2.1:
++=======
+ 2.1.16
+  * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
   * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
   * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 + * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.1.15
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d0cb200,6e82745..0835a28
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -891,12 -873,20 +895,30 @@@ public class ColumnFamilyStore implemen
      {
          synchronized (data)
          {
+             if (previousFlushFailure != null)
+                 throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                               + "no flushes can be carried out until the node is restarted.",
+                                                 previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
-             flushExecutor.execute(flush);
+             ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+             flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
              postFlushExecutor.submit(task);
-             return task;
+ 
+             @SuppressWarnings("unchecked")
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future = 
++                    // If either of the two tasks errors out, resulting future must also error out.
++                    // Combine the two futures and only return post-flush result after both have completed.
++                    Futures.transform(Futures.allAsList(flushTask, task),
++                                      new Function<List<Object>, ReplayPosition>()
++                                      {
++                                          public ReplayPosition apply(List<Object> input)
++                                          {
++                                              return (ReplayPosition) input.get(1);
++                                          }
++                                      });
+             return future;
          }
      }
  
@@@ -999,13 -978,9 +1021,12 @@@
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
 -        final ReplayPosition lastReplayPosition;
 +        final ReplayPosition commitLogUpperBound;
 +        final List<Memtable> memtables;
 +        final List<SSTableReader> readers;
-         volatile FSWriteError flushFailure = null;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
 +        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
 +                          List<Memtable> memtables, List<SSTableReader> readers)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
@@@ -1049,23 -1019,14 +1070,17 @@@
                  throw new IllegalStateException();
              }
  
-             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
 -            // must check lastReplayPosition != null because Flush may find that all memtables are clean
 -            // and so not set a lastReplayPosition
 -            if (lastReplayPosition != null)
++            CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
++            for (int i = 0 ; i < memtables.size() ; i++)
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     SSTableReader reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
-                 }
 -                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
++                Memtable memtable = memtables.get(i);
++                SSTableReader reader = readers.get(i);
++                memtable.cfs.data.permitCompactionOfFlushed(reader);
++                memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
              }
 -
              metric.pendingFlushes.dec();
 +
-             if (flushFailure != null)
-                 throw flushFailure;
 +            return commitLogUpperBound;
          }
      }
  
@@@ -1162,11 -1131,13 +1177,15 @@@
                  for (Memtable memtable : memtables)
                  {
                      // flush the memtable
 -                    MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
 +                    SSTableReader reader = memtable.flush();
 +                    memtable.cfs.data.replaceFlushed(memtable, reader);
                      reclaim(memtable);
 +                    readers.add(reader);
                  }
+ 
+                 // signal the post-flush we've done our work
+                 // Note: This should not be done in case of error. Read more below.
+                 postFlush.latch.countDown();
              }
              catch (FSWriteError e)
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
index 0000000,c0023dc..826d6e6
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@@ -1,0 -1,87 +1,95 @@@
+ /*
+  * 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.cql3;
+ 
+ import static junit.framework.Assert.fail;
+ 
+ import java.io.IOError;
+ import java.util.UUID;
+ import java.util.concurrent.ExecutionException;
+ 
++import org.junit.After;
+ import org.junit.Assert;
+ 
+ import org.apache.cassandra.db.BlacklistedDirectories;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Directories.DataDirectory;
+ import org.apache.cassandra.db.commitlog.CommitLog;
+ import org.apache.cassandra.db.commitlog.CommitLogSegment;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.io.FSWriteError;
+ 
+ /**
+  * Test that exceptions during flush are treated according to the disk failure policy.
+  * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+  */
+ public class OutOfSpaceBase extends CQLTester
+ {
+     public void makeTable() throws Throwable
+     {
+         createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+     }
+ 
+     public void markDirectoriesUnwriteable()
+     {
+         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+     }
+ 
+     public void flushAndExpectError() throws InterruptedException, ExecutionException
+     {
+         try
+         {
+             Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+             fail("FSWriteError expected.");
+         }
+         catch (ExecutionException e)
+         {
+             // Correct path.
+             Assert.assertTrue(e.getCause() instanceof FSWriteError);
+         }
+ 
+         // Make sure commit log wasn't discarded.
+         UUID cfid = currentTableMetadata().cfId;
+         for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+             if (segment.getDirtyCFIDs().contains(cfid))
+                 return;
+         fail("Expected commit log to remain dirty for the affected table.");
+     }
++
++
++    @After
++    public void afterTest() throws Throwable
++    {
++        // Override CQLTester's afterTest method; clean-up will fail due to flush failing.
++    }
+ }


[08/23] cassandra git commit: Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

* commit 'bd6654733dded3513c2c7acf96df2c364b0c043e':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-2.2
Commit: 6dc1745edd8d3861d853ee56f49ac67633a753b0
Parents: 0398521 bd66547
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:36:29 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:11 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  67 +++++---
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  95 +++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 336 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 87228d3,1275631..7fcf373
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,13 +1,58 @@@
 +2.2.8
 + * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
 + * Revert CASSANDRA-11427 (CASSANDRA-12351)
 + * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
 + * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
 + * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 + * Synchronize ThriftServer::stop() (CASSANDRA-12105)
 + * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 + * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
 + * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 + * Don't write shadowed range tombstone (CASSANDRA-12030)
 +Merged from 2.1:
++=======
+ 2.1.16
+  * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
   * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
   * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 + * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.1.15
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d0cb200,6e82745..0835a28
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -891,12 -873,20 +895,30 @@@ public class ColumnFamilyStore implemen
      {
          synchronized (data)
          {
+             if (previousFlushFailure != null)
+                 throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                               + "no flushes can be carried out until the node is restarted.",
+                                                 previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
-             flushExecutor.execute(flush);
+             ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+             flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
              postFlushExecutor.submit(task);
-             return task;
+ 
+             @SuppressWarnings("unchecked")
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future = 
++                    // If either of the two tasks errors out, resulting future must also error out.
++                    // Combine the two futures and only return post-flush result after both have completed.
++                    Futures.transform(Futures.allAsList(flushTask, task),
++                                      new Function<List<Object>, ReplayPosition>()
++                                      {
++                                          public ReplayPosition apply(List<Object> input)
++                                          {
++                                              return (ReplayPosition) input.get(1);
++                                          }
++                                      });
+             return future;
          }
      }
  
@@@ -999,13 -978,9 +1021,12 @@@
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
 -        final ReplayPosition lastReplayPosition;
 +        final ReplayPosition commitLogUpperBound;
 +        final List<Memtable> memtables;
 +        final List<SSTableReader> readers;
-         volatile FSWriteError flushFailure = null;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
 +        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
 +                          List<Memtable> memtables, List<SSTableReader> readers)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
@@@ -1049,23 -1019,14 +1070,17 @@@
                  throw new IllegalStateException();
              }
  
-             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
 -            // must check lastReplayPosition != null because Flush may find that all memtables are clean
 -            // and so not set a lastReplayPosition
 -            if (lastReplayPosition != null)
++            CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
++            for (int i = 0 ; i < memtables.size() ; i++)
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     SSTableReader reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
-                 }
 -                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
++                Memtable memtable = memtables.get(i);
++                SSTableReader reader = readers.get(i);
++                memtable.cfs.data.permitCompactionOfFlushed(reader);
++                memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
              }
 -
              metric.pendingFlushes.dec();
 +
-             if (flushFailure != null)
-                 throw flushFailure;
 +            return commitLogUpperBound;
          }
      }
  
@@@ -1162,11 -1131,13 +1177,15 @@@
                  for (Memtable memtable : memtables)
                  {
                      // flush the memtable
 -                    MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
 +                    SSTableReader reader = memtable.flush();
 +                    memtable.cfs.data.replaceFlushed(memtable, reader);
                      reclaim(memtable);
 +                    readers.add(reader);
                  }
+ 
+                 // signal the post-flush we've done our work
+                 // Note: This should not be done in case of error. Read more below.
+                 postFlush.latch.countDown();
              }
              catch (FSWriteError e)
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
index 0000000,c0023dc..826d6e6
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@@ -1,0 -1,87 +1,95 @@@
+ /*
+  * 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.cql3;
+ 
+ import static junit.framework.Assert.fail;
+ 
+ import java.io.IOError;
+ import java.util.UUID;
+ import java.util.concurrent.ExecutionException;
+ 
++import org.junit.After;
+ import org.junit.Assert;
+ 
+ import org.apache.cassandra.db.BlacklistedDirectories;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Directories.DataDirectory;
+ import org.apache.cassandra.db.commitlog.CommitLog;
+ import org.apache.cassandra.db.commitlog.CommitLogSegment;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.io.FSWriteError;
+ 
+ /**
+  * Test that exceptions during flush are treated according to the disk failure policy.
+  * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+  */
+ public class OutOfSpaceBase extends CQLTester
+ {
+     public void makeTable() throws Throwable
+     {
+         createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+     }
+ 
+     public void markDirectoriesUnwriteable()
+     {
+         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+     }
+ 
+     public void flushAndExpectError() throws InterruptedException, ExecutionException
+     {
+         try
+         {
+             Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+             fail("FSWriteError expected.");
+         }
+         catch (ExecutionException e)
+         {
+             // Correct path.
+             Assert.assertTrue(e.getCause() instanceof FSWriteError);
+         }
+ 
+         // Make sure commit log wasn't discarded.
+         UUID cfid = currentTableMetadata().cfId;
+         for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+             if (segment.getDirtyCFIDs().contains(cfid))
+                 return;
+         fail("Expected commit log to remain dirty for the affected table.");
+     }
++
++
++    @After
++    public void afterTest() throws Throwable
++    {
++        // Override CQLTester's afterTest method; clean-up will fail due to flush failing.
++    }
+ }


[10/23] cassandra git commit: Merge commit '6dc1745' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '6dc1745' into cassandra-3.0

* commit '6dc1745':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-3.0
Commit: cf85f520c768a6494281dd5e94fb12b0b07dd1b0
Parents: b66e5a1 6dc1745
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:37:43 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:43 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[11/23] cassandra git commit: Merge commit '6dc1745' into cassandra-3.0

Posted by sl...@apache.org.
Merge commit '6dc1745' into cassandra-3.0

* commit '6dc1745':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/cassandra-3.9
Commit: cf85f520c768a6494281dd5e94fb12b0b07dd1b0
Parents: b66e5a1 6dc1745
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:37:43 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:43 2016 +0200

----------------------------------------------------------------------

----------------------------------------------------------------------



[09/23] cassandra git commit: Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

Posted by sl...@apache.org.
Merge commit 'bd6654733dded3513c2c7acf96df2c364b0c043e' into cassandra-2.2

* commit 'bd6654733dded3513c2c7acf96df2c364b0c043e':
  Disable passing control to post-flush after flush failure to prevent data loss.


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

Branch: refs/heads/trunk
Commit: 6dc1745edd8d3861d853ee56f49ac67633a753b0
Parents: 0398521 bd66547
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Fri Aug 5 15:36:29 2016 +0200
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:37:11 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  67 +++++---
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  95 +++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 336 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 87228d3,1275631..7fcf373
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,55 -1,13 +1,58 @@@
 +2.2.8
 + * Release sstables of failed stream sessions only when outgoing transfers are finished (CASSANDRA-11345)
 + * Revert CASSANDRA-11427 (CASSANDRA-12351)
 + * Wait for tracing events before returning response and query at same consistency level client side (CASSANDRA-11465)
 + * cqlsh copyutil should get host metadata by connected address (CASSANDRA-11979)
 + * Fixed cqlshlib.test.remove_test_db (CASSANDRA-12214)
 + * Synchronize ThriftServer::stop() (CASSANDRA-12105)
 + * Use dedicated thread for JMX notifications (CASSANDRA-12146)
 + * NPE when trying to remove purgable tombstones from result (CASSANDRA-12143)
 + * Improve streaming synchronization and fault tolerance (CASSANDRA-11414)
 + * MemoryUtil.getShort() should return an unsigned short also for architectures not supporting unaligned memory accesses (CASSANDRA-11973)
 + * Don't write shadowed range tombstone (CASSANDRA-12030)
 +Merged from 2.1:
++=======
+ 2.1.16
+  * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
   * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
   * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
 - * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
   * Improve digest calculation in the presence of overlapping tombstones (CASSANDRA-11349)
 + * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)
 + * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
  
  
 -2.1.15
 - * Account for partition deletions in tombstone histogram (CASSANDRA-12112)
 +2.2.7
 + * Allow nodetool info to run with readonly JMX access (CASSANDRA-11755)
 + * Validate bloom_filter_fp_chance against lowest supported
 +   value when the table is created (CASSANDRA-11920)
 + * RandomAccessReader: call isEOF() only when rebuffering, not for every read operation (CASSANDRA-12013)
 + * Don't send erroneous NEW_NODE notifications on restart (CASSANDRA-11038)
 + * StorageService shutdown hook should use a volatile variable (CASSANDRA-11984)
 + * Persist local metadata earlier in startup sequence (CASSANDRA-11742)
 + * Run CommitLog tests with different compression settings (CASSANDRA-9039)
 + * cqlsh: fix tab completion for case-sensitive identifiers (CASSANDRA-11664)
 + * Avoid showing estimated key as -1 in tablestats (CASSANDRA-11587)
 + * Fix possible race condition in CommitLog.recover (CASSANDRA-11743)
 + * Enable client encryption in sstableloader with cli options (CASSANDRA-11708)
 + * Possible memory leak in NIODataInputStream (CASSANDRA-11867)
 + * Fix commit log replay after out-of-order flush completion (CASSANDRA-9669)
 + * Add seconds to cqlsh tracing session duration (CASSANDRA-11753)
 + * Prohibit Reverse Counter type as part of the PK (CASSANDRA-9395)
 + * cqlsh: correctly handle non-ascii chars in error messages (CASSANDRA-11626)
 + * Exit JVM if JMX server fails to startup (CASSANDRA-11540)
 + * Produce a heap dump when exiting on OOM (CASSANDRA-9861)
 + * Avoid read repairing purgeable tombstones on range slices (CASSANDRA-11427)
 + * Restore ability to filter on clustering columns when using a 2i (CASSANDRA-11510)
 + * JSON datetime formatting needs timezone (CASSANDRA-11137)
 + * Fix is_dense recalculation for Thrift-updated tables (CASSANDRA-11502)
 + * Remove unnescessary file existence check during anticompaction (CASSANDRA-11660)
 + * Add missing files to debian packages (CASSANDRA-11642)
 + * Avoid calling Iterables::concat in loops during ModificationStatement::getFunctions (CASSANDRA-11621)
 + * cqlsh: COPY FROM should use regular inserts for single statement batches and
 +   report errors correctly if workers processes crash on initialization (CASSANDRA-11474)
 + * Always close cluster with connection in CqlRecordWriter (CASSANDRA-11553)
 + * Fix slice queries on ordered COMPACT tables (CASSANDRA-10988)
 +Merged from 2.1:
   * Avoid stalling paxos when the paxos state expires (CASSANDRA-12043)
   * Remove finished incoming streaming connections from MessagingService (CASSANDRA-11854)
   * Don't try to get sstables for non-repairing column families (CASSANDRA-12077)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index d0cb200,6e82745..0835a28
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -891,12 -873,20 +895,30 @@@ public class ColumnFamilyStore implemen
      {
          synchronized (data)
          {
+             if (previousFlushFailure != null)
+                 throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                               + "no flushes can be carried out until the node is restarted.",
+                                                 previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
-             flushExecutor.execute(flush);
+             ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+             flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
              postFlushExecutor.submit(task);
-             return task;
+ 
+             @SuppressWarnings("unchecked")
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future = 
++                    // If either of the two tasks errors out, resulting future must also error out.
++                    // Combine the two futures and only return post-flush result after both have completed.
++                    Futures.transform(Futures.allAsList(flushTask, task),
++                                      new Function<List<Object>, ReplayPosition>()
++                                      {
++                                          public ReplayPosition apply(List<Object> input)
++                                          {
++                                              return (ReplayPosition) input.get(1);
++                                          }
++                                      });
+             return future;
          }
      }
  
@@@ -999,13 -978,9 +1021,12 @@@
          final boolean flushSecondaryIndexes;
          final OpOrder.Barrier writeBarrier;
          final CountDownLatch latch = new CountDownLatch(1);
 -        final ReplayPosition lastReplayPosition;
 +        final ReplayPosition commitLogUpperBound;
 +        final List<Memtable> memtables;
 +        final List<SSTableReader> readers;
-         volatile FSWriteError flushFailure = null;
  
 -        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
 +        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition commitLogUpperBound,
 +                          List<Memtable> memtables, List<SSTableReader> readers)
          {
              this.writeBarrier = writeBarrier;
              this.flushSecondaryIndexes = flushSecondaryIndexes;
@@@ -1049,23 -1019,14 +1070,17 @@@
                  throw new IllegalStateException();
              }
  
-             // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-             if (flushFailure == null)
 -            // must check lastReplayPosition != null because Flush may find that all memtables are clean
 -            // and so not set a lastReplayPosition
 -            if (lastReplayPosition != null)
++            CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
++            for (int i = 0 ; i < memtables.size() ; i++)
              {
-                 CommitLog.instance.discardCompletedSegments(metadata.cfId, commitLogUpperBound);
-                 for (int i = 0 ; i < memtables.size() ; i++)
-                 {
-                     Memtable memtable = memtables.get(i);
-                     SSTableReader reader = readers.get(i);
-                     memtable.cfs.data.permitCompactionOfFlushed(reader);
-                     memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
-                 }
 -                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
++                Memtable memtable = memtables.get(i);
++                SSTableReader reader = readers.get(i);
++                memtable.cfs.data.permitCompactionOfFlushed(reader);
++                memtable.cfs.compactionStrategyWrapper.replaceFlushed(memtable, reader);
              }
 -
              metric.pendingFlushes.dec();
 +
-             if (flushFailure != null)
-                 throw flushFailure;
 +            return commitLogUpperBound;
          }
      }
  
@@@ -1162,11 -1131,13 +1177,15 @@@
                  for (Memtable memtable : memtables)
                  {
                      // flush the memtable
 -                    MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
 +                    SSTableReader reader = memtable.flush();
 +                    memtable.cfs.data.replaceFlushed(memtable, reader);
                      reclaim(memtable);
 +                    readers.add(reader);
                  }
+ 
+                 // signal the post-flush we've done our work
+                 // Note: This should not be done in case of error. Read more below.
+                 postFlush.latch.countDown();
              }
              catch (FSWriteError e)
              {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6dc1745e/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
index 0000000,c0023dc..826d6e6
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@@ -1,0 -1,87 +1,95 @@@
+ /*
+  * 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.cql3;
+ 
+ import static junit.framework.Assert.fail;
+ 
+ import java.io.IOError;
+ import java.util.UUID;
+ import java.util.concurrent.ExecutionException;
+ 
++import org.junit.After;
+ import org.junit.Assert;
+ 
+ import org.apache.cassandra.db.BlacklistedDirectories;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.Directories.DataDirectory;
+ import org.apache.cassandra.db.commitlog.CommitLog;
+ import org.apache.cassandra.db.commitlog.CommitLogSegment;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.io.FSWriteError;
+ 
+ /**
+  * Test that exceptions during flush are treated according to the disk failure policy.
+  * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+  */
+ public class OutOfSpaceBase extends CQLTester
+ {
+     public void makeTable() throws Throwable
+     {
+         createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+ 
+         for (int i = 0; i < 10; i++)
+             execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+     }
+ 
+     public void markDirectoriesUnwriteable()
+     {
+         ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+         try
+         {
+             for ( ; ; )
+             {
+                 DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                 BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+             }
+         }
+         catch (IOError e)
+         {
+             // Expected -- marked all directories as unwritable
+         }
+     }
+ 
+     public void flushAndExpectError() throws InterruptedException, ExecutionException
+     {
+         try
+         {
+             Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+             fail("FSWriteError expected.");
+         }
+         catch (ExecutionException e)
+         {
+             // Correct path.
+             Assert.assertTrue(e.getCause() instanceof FSWriteError);
+         }
+ 
+         // Make sure commit log wasn't discarded.
+         UUID cfid = currentTableMetadata().cfId;
+         for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+             if (segment.getDirtyCFIDs().contains(cfid))
+                 return;
+         fail("Expected commit log to remain dirty for the affected table.");
+     }
++
++
++    @After
++    public void afterTest() throws Throwable
++    {
++        // Override CQLTester's afterTest method; clean-up will fail due to flush failing.
++    }
+ }


[02/23] cassandra git commit: Disable passing control to post-flush after flush failure to prevent data loss.

Posted by sl...@apache.org.
Disable passing control to post-flush after flush failure to prevent
data loss.

patch by Branimir Lambov; reviewed by Sylvain Lebresne for
CASSANDRA-11828

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

Branch: refs/heads/cassandra-2.2
Commit: bd6654733dded3513c2c7acf96df2c364b0c043e
Parents: bc0d1da
Author: Branimir Lambov <br...@datastax.com>
Authored: Wed Aug 3 11:32:48 2016 +0300
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Fri Aug 5 15:35:25 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  45 ++++--
 .../apache/cassandra/cql3/OutOfSpaceBase.java   |  87 ++++++++++
 .../cassandra/cql3/OutOfSpaceDieTest.java       |  68 ++++++++
 .../cassandra/cql3/OutOfSpaceIgnoreTest.java    |  60 +++++++
 .../cassandra/cql3/OutOfSpaceStopTest.java      |  63 ++++++++
 .../apache/cassandra/cql3/OutOfSpaceTest.java   | 157 -------------------
 7 files changed, 311 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 8ecc787..1275631 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.16
+ * Disable passing control to post-flush after flush failure to prevent data loss (CASSANDRA-11828)
  * Allow STCS-in-L0 compactions to reduce scope with LCS (CASSANDRA-12040)
  * cannot use cql since upgrading python to 2.7.11+ (CASSANDRA-11850)
  * Fix filtering on clustering columns when 2i is used (CASSANDRA-11907)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/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 b64d5de..6e82745 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -99,6 +99,10 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                                                                                               new NamedThreadFactory("MemtablePostFlush"),
                                                                                               "internal");
 
+    // If a flush fails with an error the post-flush is never allowed to continue. This stores the error that caused it
+    // to be able to show an error on following flushes instead of blindly continuing.
+    private static volatile FSWriteError previousFlushFailure = null;
+
     private static final ExecutorService reclaimExecutor = new JMXEnabledThreadPoolExecutor(1,
                                                                                             StageManager.KEEPALIVE,
                                                                                             TimeUnit.SECONDS,
@@ -869,12 +873,20 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     {
         synchronized (data)
         {
+            if (previousFlushFailure != null)
+                throw new IllegalStateException("A flush previously failed with the error below. To prevent data loss, "
+                                              + "no flushes can be carried out until the node is restarted.",
+                                                previousFlushFailure);
             logFlush();
             Flush flush = new Flush(false);
-            flushExecutor.execute(flush);
+            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
+            flushExecutor.submit(flushTask);
             ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
             postFlushExecutor.submit(task);
-            return task;
+
+            @SuppressWarnings("unchecked")
+            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
+            return future;
         }
     }
 
@@ -967,7 +979,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         final OpOrder.Barrier writeBarrier;
         final CountDownLatch latch = new CountDownLatch(1);
         final ReplayPosition lastReplayPosition;
-        volatile FSWriteError flushFailure = null;
 
         private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier, ReplayPosition lastReplayPosition)
         {
@@ -1010,16 +1021,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
             // must check lastReplayPosition != null because Flush may find that all memtables are clean
             // and so not set a lastReplayPosition
-            // If a flush errored out but the error was ignored, make sure we don't discard the commit log.
-            if (lastReplayPosition != null && flushFailure == null)
+            if (lastReplayPosition != null)
             {
                 CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
             }
 
             metric.pendingFlushes.dec();
-
-            if (flushFailure != null)
-                throw flushFailure;
         }
     }
 
@@ -1127,16 +1134,28 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
                     reclaim(memtable);
                 }
+
+                // signal the post-flush we've done our work
+                // Note: This should not be done in case of error. Read more below.
+                postFlush.latch.countDown();
             }
             catch (FSWriteError e)
             {
                 JVMStabilityInspector.inspectThrowable(e);
-                // If we weren't killed, try to continue work but do not allow CommitLog to be discarded.
-                postFlush.flushFailure = e;
+                // The call above may kill the process or the transports, or ignore the error.
+                // In any case we should not be passing on control to post-flush as a subsequent succeeding flush
+                // could mask the error and:
+                //   - let the commit log discard unpersisted data, resulting in data loss
+                //   - let truncations proceed, with the possibility of resurrecting the unflushed data
+                //   - let snapshots succeed with incomplete data
+
+                // Not passing control on means that all flushes from the moment of failure cannot complete
+                // (including snapshots).
+                // If the disk failure policy is ignore, this will cause memtables and the commit log to grow
+                // unboundedly until the node eventually fails.
+                previousFlushFailure = e;
+                throw e;
             }
-
-            // signal the post-flush we've done our work
-            postFlush.latch.countDown();
         }
 
         private void reclaim(final Memtable memtable)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
new file mode 100644
index 0000000..c0023dc
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceBase.java
@@ -0,0 +1,87 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import java.io.IOError;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.junit.Assert;
+
+import org.apache.cassandra.db.BlacklistedDirectories;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories.DataDirectory;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.db.commitlog.CommitLogSegment;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.io.FSWriteError;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ * We cannot recover after a failed flush due to postFlushExecutor being stuck, so each test needs to run separately.
+ */
+public class OutOfSpaceBase extends CQLTester
+{
+    public void makeTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
+
+        for (int i = 0; i < 10; i++)
+            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
+    }
+
+    public void markDirectoriesUnwriteable()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
+        try
+        {
+            for ( ; ; )
+            {
+                DataDirectory dir = cfs.directories.getWriteableLocation(1);
+                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
+            }
+        }
+        catch (IOError e)
+        {
+            // Expected -- marked all directories as unwritable
+        }
+    }
+
+    public void flushAndExpectError() throws InterruptedException, ExecutionException
+    {
+        try
+        {
+            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
+            fail("FSWriteError expected.");
+        }
+        catch (ExecutionException e)
+        {
+            // Correct path.
+            Assert.assertTrue(e.getCause() instanceof FSWriteError);
+        }
+
+        // Make sure commit log wasn't discarded.
+        UUID cfid = currentTableMetadata().cfId;
+        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
+            if (segment.getDirtyCFIDs().contains(cfid))
+                return;
+        fail("Expected commit log to remain dirty for the affected table.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
new file mode 100644
index 0000000..46d71e4
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceDieTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+import org.apache.cassandra.utils.KillerForTests;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceDieTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableDie() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        KillerForTests killerForTests = new KillerForTests();
+        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
+            flushAndExpectError();
+            Assert.assertTrue(killerForTests.wasKilled());
+            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+            JVMStabilityInspector.replaceKiller(originalKiller);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
new file mode 100644
index 0000000..854de80
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceIgnoreTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceIgnoreTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableIgnore() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
+            flushAndExpectError();
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.java
new file mode 100644
index 0000000..b48df56
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/OutOfSpaceStopTest.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.cql3;
+
+import static junit.framework.Assert.fail;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.gms.Gossiper;
+
+/**
+ * Test that exceptions during flush are treated according to the disk failure policy.
+ */
+public class OutOfSpaceStopTest extends OutOfSpaceBase
+{
+    @Test
+    public void testFlushUnwriteableStop() throws Throwable
+    {
+        makeTable();
+        markDirectoriesUnwriteable();
+
+        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
+        try
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
+            flushAndExpectError();
+            Assert.assertFalse(Gossiper.instance.isEnabled());
+        }
+        finally
+        {
+            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
+        }
+
+        makeTable();
+        try
+        {
+            flush();
+            fail("Subsequent flushes expected to fail.");
+        }
+        catch (RuntimeException e)
+        {
+            // correct path
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bd665473/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java b/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
deleted file mode 100644
index 8304aff..0000000
--- a/test/unit/org/apache/cassandra/cql3/OutOfSpaceTest.java
+++ /dev/null
@@ -1,157 +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.cql3;
-
-import static junit.framework.Assert.fail;
-
-import java.io.IOError;
-import java.util.UUID;
-import java.util.concurrent.ExecutionException;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.config.Config.DiskFailurePolicy;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.BlacklistedDirectories;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories.DataDirectory;
-import org.apache.cassandra.db.commitlog.CommitLog;
-import org.apache.cassandra.db.commitlog.CommitLogSegment;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.utils.JVMStabilityInspector;
-import org.apache.cassandra.utils.KillerForTests;
-
-/**
- * Test that TombstoneOverwhelmingException gets thrown when it should be and doesn't when it shouldn't be.
- */
-public class OutOfSpaceTest extends CQLTester
-{
-    @Test
-    public void testFlushUnwriteableDie() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        KillerForTests killerForTests = new KillerForTests();
-        JVMStabilityInspector.Killer originalKiller = JVMStabilityInspector.replaceKiller(killerForTests);
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.die);
-            flushAndExpectError();
-            Assert.assertTrue(killerForTests.wasKilled());
-            Assert.assertFalse(killerForTests.wasKilledQuietly()); //only killed quietly on startup failure
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-            JVMStabilityInspector.replaceKiller(originalKiller);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableStop() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.stop);
-            flushAndExpectError();
-            Assert.assertFalse(Gossiper.instance.isEnabled());
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-    }
-
-    @Test
-    public void testFlushUnwriteableIgnore() throws Throwable
-    {
-        makeTable();
-        markDirectoriesUnwriteable();
-
-        DiskFailurePolicy oldPolicy = DatabaseDescriptor.getDiskFailurePolicy();
-        try
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(DiskFailurePolicy.ignore);
-            flushAndExpectError();
-        }
-        finally
-        {
-            DatabaseDescriptor.setDiskFailurePolicy(oldPolicy);
-        }
-
-        // Next flush should succeed.
-        makeTable();
-        flush();
-    }
-
-    public void makeTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (a text, b text, c text, PRIMARY KEY (a, b));");
-
-        // insert exactly the amount of tombstones that shouldn't trigger an exception
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s (a, b, c) VALUES ('key', 'column" + i + "', null);");
-    }
-
-    public void markDirectoriesUnwriteable()
-    {
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentTable());
-        try
-        {
-            for ( ; ; )
-            {
-                DataDirectory dir = cfs.directories.getWriteableLocation(1);
-                BlacklistedDirectories.maybeMarkUnwritable(cfs.directories.getLocationForDisk(dir));
-            }
-        }
-        catch (IOError e)
-        {
-            // Expected -- marked all directories as unwritable
-        }
-    }
-
-    public void flushAndExpectError() throws InterruptedException, ExecutionException
-    {
-        try
-        {
-            Keyspace.open(KEYSPACE).getColumnFamilyStore(currentTable()).forceFlush().get();
-            fail("FSWriteError expected.");
-        }
-        catch (ExecutionException e)
-        {
-            // Correct path.
-            Assert.assertTrue(e.getCause() instanceof FSWriteError);
-        }
-
-        // Make sure commit log wasn't discarded.
-        UUID cfid = currentTableMetadata().cfId;
-        for (CommitLogSegment segment : CommitLog.instance.allocator.getActiveSegments())
-            if (segment.getDirtyCFIDs().contains(cfid))
-                return;
-        fail("Expected commit log to remain dirty for the affected table.");
-    }
-}