cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Anuj Wadehra <>
Subject Re: Huge number of sstables after adding server to existing cluster
Date Sun, 05 Apr 2015 05:09:32 GMT
We faced compaction issue with SCTS in 2.0.3. Till we upgrade, we added a dummy read every
1000 writes as workaround . Compaction started happenning in Write only heavy loads.

Anuj Wadehra 

From:"graham sanderson" <>
Date:Sun, 5 Apr, 2015 at 9:35 am
Subject:Re: Huge number of sstables after adding server to existing cluster

I have not thought thru why adding a node would cause this behavior, but

related issues (which end up with causing excessive numbers of sstables) - we saw many thousands
per node for some tables

If you have a manageable number of tables, try setting coldReadsToOmit (a setting on SizeTieredCompactionStrategy)
back to 0 which was the default in 2.0.x

Otherwise you could apply this patch (which reverts the default - and I doubt you had overridden
it), but note that the coldReadsToOmit is fixed in 2.1.4, so if you can do it just by chaining
table config then that is good.

diff --git a/src/java/org/apache/cassandra/db/compaction/

index fbd715c..cbb8c8b 100644

--- a/src/java/org/apache/cassandra/db/compaction/

+++ b/src/java/org/apache/cassandra/db/compaction/

@@ -118,7 +118,11 @@ public class SizeTieredCompactionStrategy extends AbstractCompactionStrategy

     static List<SSTableReader> filterColdSSTables(List<SSTableReader> sstables,
double coldReadsToOmit, int minThreshold)


         if (coldReadsToOmit == 0.0)

+        {

+            if (!sstables.isEmpty())

+                logger.debug("Skipping cold sstable filter for list sized {} containing
{}", sstables.size(), sstables.get(0).getFilename());

             return sstables;

+        }


         // Sort the sstables by hotness (coldest-first). We first build a map because
the hotness may change during the sort.

         final Map<SSTableReader, Double> hotnessSnapshot = getHotnessMap(sstables);

diff --git a/src/java/org/apache/cassandra/db/compaction/

index 84e7d61..c6c5f1b 100644

--- a/src/java/org/apache/cassandra/db/compaction/

+++ b/src/java/org/apache/cassandra/db/compaction/

@@ -26,7 +26,7 @@ public final class SizeTieredCompactionStrategyOptions

     protected static final long DEFAULT_MIN_SSTABLE_SIZE = 50L * 1024L * 1024L;

     protected static final double DEFAULT_BUCKET_LOW = 0.5;

     protected static final double DEFAULT_BUCKET_HIGH = 1.5;

-    protected static final double DEFAULT_COLD_READS_TO_OMIT = 0.05;

+    protected static final double DEFAULT_COLD_READS_TO_OMIT = 0.0;

     protected static final String MIN_SSTABLE_SIZE_KEY = "min_sstable_size";

     protected static final String BUCKET_LOW_KEY = "bucket_low";

     protected static final String BUCKET_HIGH_KEY = "bucket_high";

On Apr 4, 2015, at 4:23 PM, Mantas Klasavičius <> wrote:

Thanks a lot for all to your responses

I should mention we are running 2.1.3 and  I have set setcompactionthroughput 0 already 

nodetool enableautocompaction keyspace table command/bug is new to me I will definitely will
try this out and let you know

One more thing I wan't to clarify did I understand correctly 32 is the max number for sstables
for normally operating cassandra node?

Best regards


On Sat, Apr 4, 2015 at 4:47 AM, graham sanderson <> wrote:

As does 2.1.3

On Apr 3, 2015, at 5:36 PM, Robert Coli <> wrote:

On Fri, Apr 3, 2015 at 1:04 PM, Thomas Borg Salling <> wrote:

I agree with Pranay. I have experienced exactly the same on C* 2.1.2.

2.1.2 had a serious bug which resulted in extra files, which is different from the overall
issue I am referring to.



View raw message