You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2016/01/22 07:35:06 UTC

[03/10] cassandra git commit: Run STCS in L0 even when the higher levels do not produce a candidate

Run STCS in L0 even when the higher levels do not produce a candidate

Patch by Carl Yeksigian; reviewed by marcuse for CASSANDRA-10979


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

Branch: refs/heads/cassandra-3.3
Commit: 38dde32ef40251c2d3dcf7015521c60c28c5c426
Parents: 968b134
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jan 20 15:51:06 2016 -0500
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Fri Jan 22 07:23:24 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 ++
 .../db/compaction/LeveledManifest.java          | 34 ++++++++++++++------
 2 files changed, 26 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/38dde32e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ae3082a..751267e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.2.5
+ * Start L0 STCS-compactions even if there is a L0 -> L1 compaction
+   going (CASSANDRA-10979)
  * Make UUID LSB unique per process (CASSANDRA-7925)
  * Avoid NPE when performing sstable tasks (scrub etc.) (CASSANDRA-10980)
  * Make sure client gets tombstone overwhelmed warning (CASSANDRA-9465)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/38dde32e/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index d90318f..544b0a8 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -322,15 +322,9 @@ public class LeveledManifest
             if (score > 1.001)
             {
                 // before proceeding with a higher level, let's see if L0 is far enough behind to warrant STCS
-                if (!DatabaseDescriptor.getDisableSTCSInL0() && getLevel(0).size() > MAX_COMPACTING_L0)
-                {
-                    List<SSTableReader> mostInteresting = getSSTablesForSTCS(getLevel(0));
-                    if (!mostInteresting.isEmpty())
-                    {
-                        logger.trace("L0 is too far behind, performing size-tiering there first");
-                        return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE);
-                    }
-                }
+                CompactionCandidate l0Compaction = getSTCSInL0CompactionCandidate();
+                if (l0Compaction != null)
+                    return l0Compaction;
 
                 // L0 is fine, proceed with this level
                 Collection<SSTableReader> candidates = getCandidatesFor(i);
@@ -354,10 +348,30 @@ public class LeveledManifest
             return null;
         Collection<SSTableReader> candidates = getCandidatesFor(0);
         if (candidates.isEmpty())
-            return null;
+        {
+            // Since we don't have any other compactions to do, see if there is a STCS compaction to perform in L0; if
+            // there is a long running compaction, we want to make sure that we continue to keep the number of SSTables
+            // small in L0.
+            return getSTCSInL0CompactionCandidate();
+        }
         return new CompactionCandidate(candidates, getNextLevel(candidates), cfs.getCompactionStrategy().getMaxSSTableBytes());
     }
 
+    private CompactionCandidate getSTCSInL0CompactionCandidate()
+    {
+        if (!DatabaseDescriptor.getDisableSTCSInL0() && getLevel(0).size() > MAX_COMPACTING_L0)
+        {
+            List<SSTableReader> mostInteresting = getSSTablesForSTCS(getLevel(0));
+            if (!mostInteresting.isEmpty())
+            {
+                logger.debug("L0 is too far behind, performing size-tiering there first");
+                return new CompactionCandidate(mostInteresting, 0, Long.MAX_VALUE);
+            }
+        }
+
+        return null;
+    }
+
     private List<SSTableReader> getSSTablesForSTCS(Collection<SSTableReader> sstables)
     {
         Iterable<SSTableReader> candidates = cfs.getTracker().getUncompacting(sstables);