You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by William Saar <Wi...@king.com> on 2015/01/15 20:20:57 UTC

Reasons for nodes not compacting?

Cassandra 2.1.2 with size-tiered compaction worked well during an initial test run when data was first written to the cluster, but during the second run when the initial data got overwritten we noticed that two nodes stopped compacting altogether and the number of SSTables grew dramatically. What could cause the nodes to stop its compactions?

We managed to get the non-compacting nodes to start a few compactions by lowering the thresholds, but most nodes still had 100+ files of identical size 384M that the compaction simply would not touch even when there was no load on the server. Could these files be data from the first run that had been overwritten, and is there a way to start cleaning those files before they fill up the disk? (the cluster eventually went above 50% disk usage)

We see (running compactions x 100)% CPU usage with the remaining cores free and with disk utilization still quite low, so our compactions seem CPU bound. This means manual compaction is not an option as it only seems to use 1 CPU and takes over 24 hours to complete. We increased disk utilization a lot by disabling compression and switching to size-tiered instead of leveled compaction, but it would be nice to know how to get Cassandra to start more compactions. We have set 8 concurrent compactors, but we saw that some nodes were only using 3 even though they had plenty of files of the same size...

I noticed that the compaction threads that were mostly executing CompactionController.maxPurgeableTimestamp and IntervalNode.searchInternal. We use bigint partition keys and int clustering keys, perhaps there is a way to specify a special comparator or disable clustering ordering to make compactions run faster? Maybe it is faster if we add the date to the key and issue a separate delete on the key with the old date on insertions (would perhaps allow switching to the DateTiered strategy)?

Any clues are appreciated, thanks!

William



Re: Reasons for nodes not compacting?

Posted by Tyler Hobbs <ty...@datastax.com>.
By chance, are you not performing any reads on that table, only writes?  If
you are performing reads, what sorts of reads are you doing?

If you're not doing any reads, please try altering the compaction strategy
options on that table as follows:

ALTER TABLE <tablename> WITH compaction = {'class':
'SizeTieredCompactionStrategy', 'min_threshold': '4', 'max_threshold':
'32', 'cold_reads_to_omit': 0.0};

I suspect that the code to handle "cold" sstables during compaction is
not handling your case of performing overwrites with no reads
properly. I will open a ticket to investigate this, but in the
meantime, if that is the problem, the above command should fix it.


On Thu, Jan 15, 2015 at 1:20 PM, William Saar <Wi...@king.com> wrote:

>  Cassandra 2.1.2 with size-tiered compaction worked well during an
> initial test run when data was first written to the cluster, but during the
> second run when the initial data got overwritten we noticed that two nodes
> stopped compacting altogether and the number of SSTables grew dramatically.
> What could cause the nodes to stop its compactions?
>
> We managed to get the non-compacting nodes to start a few compactions by
> lowering the thresholds, but most nodes still had 100+ files of identical
> size 384M that the compaction simply would not touch even when there was no
> load on the server. Could these files be data from the first run that had
> been overwritten, and is there a way to start cleaning those files before
> they fill up the disk? (the cluster eventually went above 50% disk usage)
>
> We see (running compactions x 100)% CPU usage with the remaining cores
> free and with disk utilization still quite low, so our compactions seem CPU
> bound. This means manual compaction is not an option as it only seems to
> use 1 CPU and takes over 24 hours to complete. We increased disk
> utilization a lot by disabling compression and switching to size-tiered
> instead of leveled compaction, but it would be nice to know how to get
> Cassandra to start more compactions. We have set 8 concurrent compactors,
> but we saw that some nodes were only using 3 even though they had plenty of
> files of the same size...
>
> I noticed that the compaction threads that were mostly executing
> CompactionController.maxPurgeableTimestamp and IntervalNode.searchInternal.
> We use bigint partition keys and int clustering keys, perhaps there is a
> way to specify a special comparator or disable clustering ordering to make
> compactions run faster? Maybe it is faster if we add the date to the key
> and issue a separate delete on the key with the old date on insertions
> (would perhaps allow switching to the DateTiered strategy)?
>
> Any clues are appreciated, thanks!
>
> William
>
>
>


-- 
Tyler Hobbs
DataStax <http://datastax.com/>