You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pa...@apache.org on 2017/12/12 19:49:23 UTC

[03/10] cassandra git commit: Fix race that prevents submitting compaction for a table when executor is full

Fix race that prevents submitting compaction for a table when executor is full

Patch by Dimitar Dimitrov; Reviewed by Paulo Motta for CASSANDRA-13801


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

Branch: refs/heads/cassandra-3.11
Commit: 35e6d61361e699908d73c277da7d9ac3390f6e5d
Parents: 02aba73
Author: dimitarndimitrov <dm...@gmail.com>
Authored: Wed Dec 6 00:53:00 2017 +0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Wed Dec 13 06:40:24 2017 +1100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/compaction/CompactionManager.java        | 15 ++++--
 .../db/compaction/CompactionsBytemanTest.java   | 56 ++++++++++++++++++++
 3 files changed, 68 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/35e6d613/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5200eb1..33fd264 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.12
+ * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
  * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
  * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35e6d613/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 2e69b6f..3350b20 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -111,7 +111,8 @@ public class CompactionManager implements CompactionManagerMBean
     private final static CompactionExecutor cacheCleanupExecutor = new CacheCleanupExecutor();
 
     private final CompactionMetrics metrics = new CompactionMetrics(executor, validationExecutor);
-    private final Multiset<ColumnFamilyStore> compactingCF = ConcurrentHashMultiset.create();
+    @VisibleForTesting
+    final Multiset<ColumnFamilyStore> compactingCF = ConcurrentHashMultiset.create();
 
     private final RateLimiter compactionRateLimiter = RateLimiter.create(Double.MAX_VALUE);
 
@@ -155,6 +156,12 @@ public class CompactionManager implements CompactionManagerMBean
             return Collections.emptyList();
         }
 
+        /**
+         * If a CF is currently being compacted, and there are no idle threads, submitBackground should be a no-op;
+         * we can wait for the current compaction to finish and re-submit when more information is available.
+         * Otherwise, we should submit at least one task to prevent starvation by busier CFs, and more if there
+         * are idle threads stil. (CASSANDRA-4310)
+         */
         int count = compactingCF.count(cfs);
         if (count > 0 && executor.getActiveCount() >= executor.getMaximumPoolSize())
         {
@@ -171,10 +178,9 @@ public class CompactionManager implements CompactionManagerMBean
         List<Future<?>> futures = new ArrayList<>(1);
         Future<?> fut = executor.submitIfRunning(new BackgroundCompactionCandidate(cfs), "background task");
         if (!fut.isCancelled())
-        {
-            compactingCF.add(cfs);
             futures.add(fut);
-        }
+        else
+            compactingCF.remove(cfs);
         return futures;
     }
 
@@ -233,6 +239,7 @@ public class CompactionManager implements CompactionManagerMBean
 
         BackgroundCompactionCandidate(ColumnFamilyStore cfs)
         {
+            compactingCF.add(cfs);
             this.cfs = cfs;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/35e6d613/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java
new file mode 100644
index 0000000..0b391a5
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsBytemanTest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.compaction;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.utils.FBUtilities;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
+
+import static org.junit.Assert.assertEquals;
+
+@RunWith(BMUnitRunner.class)
+public class CompactionsBytemanTest extends CQLTester
+{
+    @Test
+    @BMRule(name = "Delay background compaction task future check",
+            targetClass = "CompactionManager",
+            targetMethod = "submitBackground",
+            targetLocation = "AT INVOKE java.util.concurrent.Future.isCancelled",
+            condition = "!$cfs.keyspace.getName().contains(\"system\")",
+            action = "Thread.sleep(1000)")
+    public void testCompactingCFCounting() throws Throwable
+    {
+        String table = createTable("CREATE TABLE %s (k INT, c INT, v INT, PRIMARY KEY (k, c))");
+        ColumnFamilyStore cfs = Keyspace.open(CQLTester.KEYSPACE).getColumnFamilyStore(table);
+        cfs.enableAutoCompaction();
+
+        execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 1, 1);
+        assertEquals(0, CompactionManager.instance.compactingCF.count(cfs));
+        cfs.forceBlockingFlush();
+
+        FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(cfs));
+        assertEquals(0, CompactionManager.instance.compactingCF.count(cfs));
+    }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org