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.");
- }
-}