You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2014/08/27 17:07:41 UTC

[1/2] git commit: Avoid recycling CL segments until all writes have been flushed for all keyspaces in segment

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 5abd129fd -> ec0975304


Avoid recycling CL segments until all writes have been flushed for all keyspaces in segment

Patch by belliottsmith; reviewed by tjake for CASSANDRA-7437


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

Branch: refs/heads/cassandra-2.1
Commit: ce3fcbe5f810cf1ebe0d86b702996b71e3ba03ec
Parents: 049762b
Author: Jake Luciani <ja...@apache.org>
Authored: Wed Aug 27 10:59:14 2014 -0400
Committer: Jake Luciani <ja...@apache.org>
Committed: Wed Aug 27 10:59:14 2014 -0400

----------------------------------------------------------------------
 .../db/commitlog/CommitLogSegmentManager.java   | 15 ++++
 .../cql3/DropKeyspaceCommitLogRecycleTest.java  | 91 ++++++++++++++++++++
 2 files changed, 106 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce3fcbe5/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 e1a7e39..05b3aa5 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -21,9 +21,11 @@ import java.io.File;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
@@ -292,8 +294,21 @@ public class CommitLogSegmentManager
         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
+        Set<Keyspace> keyspaces = new HashSet<>();
+        for (UUID cfId : last.getDirtyCFIDs())
+        {
+            ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(cfId);
+            if (cfs != null)
+                keyspaces.add(cfs.keyspace);
+        }
+        for (Keyspace keyspace : keyspaces)
+            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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/ce3fcbe5/test/long/org/apache/cassandra/cql3/DropKeyspaceCommitLogRecycleTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/DropKeyspaceCommitLogRecycleTest.java b/test/long/org/apache/cassandra/cql3/DropKeyspaceCommitLogRecycleTest.java
new file mode 100644
index 0000000..a0bacea
--- /dev/null
+++ b/test/long/org/apache/cassandra/cql3/DropKeyspaceCommitLogRecycleTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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 org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.SchemaLoader;
+
+import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
+
+/**
+ * Base class for CQL tests.
+ */
+public class DropKeyspaceCommitLogRecycleTest
+{
+    protected static final Logger logger = LoggerFactory.getLogger(DropKeyspaceCommitLogRecycleTest.class);
+
+    private static final String KEYSPACE = "cql_test_keyspace";
+    private static final String KEYSPACE2 = "cql_test_keyspace2";
+
+    static
+    {
+        // Once per-JVM is enough
+        SchemaLoader.prepareServer();
+    }
+
+    private void create(boolean both)
+    {
+        executeOnceInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE));
+        executeOnceInternal(String.format("CREATE TABLE %s.test (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))", KEYSPACE));
+        
+        if (both)
+        {
+            executeOnceInternal(String.format("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}", KEYSPACE2));
+            executeOnceInternal(String.format("CREATE TABLE %s.test (k1 int, k2 int, v int, PRIMARY KEY (k1, k2))", KEYSPACE2));
+        }
+    }
+
+    private void insert()
+    {
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (0, 0, 0)", KEYSPACE));
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (1, 1, 1)", KEYSPACE));
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (2, 2, 2)", KEYSPACE));
+
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (0, 0, 0)", KEYSPACE2));
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (1, 1, 1)", KEYSPACE2));
+        executeOnceInternal(String.format("INSERT INTO %s.test (k1, k2, v) VALUES (2, 2, 2)", KEYSPACE2));       
+    }
+
+    private void drop(boolean both)
+    {
+        executeOnceInternal(String.format("DROP KEYSPACE IF EXISTS %s", KEYSPACE));
+        if (both)
+            executeOnceInternal(String.format("DROP KEYSPACE IF EXISTS %s", KEYSPACE2));
+    }
+
+    @Test
+    public void testRecycle()
+    {
+        for (int i = 0 ; i < 1000 ; i++)
+        {
+            create(i == 0);
+            insert();
+            drop(false);
+        }
+    }
+
+    @After
+    public void afterTest() throws Throwable
+    {
+        drop(true);
+    }
+}


[2/2] git commit: Merge branch 'cassandra-2.1.0' into cassandra-2.1

Posted by ja...@apache.org.
Merge branch 'cassandra-2.1.0' into cassandra-2.1


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

Branch: refs/heads/cassandra-2.1
Commit: ec0975304c9288c800999937c249e12e397e7978
Parents: 5abd129 ce3fcbe
Author: Jake Luciani <ja...@apache.org>
Authored: Wed Aug 27 11:07:24 2014 -0400
Committer: Jake Luciani <ja...@apache.org>
Committed: Wed Aug 27 11:07:24 2014 -0400

----------------------------------------------------------------------
 .../db/commitlog/CommitLogSegmentManager.java   | 15 ++++
 .../cql3/DropKeyspaceCommitLogRecycleTest.java  | 91 ++++++++++++++++++++
 2 files changed, 106 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/ec097530/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------