[
https://issues.apache.org/jira/browse/CASSANDRA-11831?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15293756#comment-15293756
]
Wei Deng commented on CASSANDRA-11831:
--------------------------------------
I tried to reproduce this problem using an artificially created "excessive
amount of SSTable" environment. It is a one-node cluster on GCE with modest
amount of hardware resource (8GB RAM + 2 cores) running C* 2.1.14. To allow
tiny SSTables to accumulate I used "nodetool disableautocompaction" to
temporarily disable compaction when I'm inserting my test data; from the same
python code doing the insert I also perform a system call "nodetool flush"
after every insert, so that the amount of sstables can quickly grow to a number
that's usually hard to get on a tiny node. The table I'm inserting into has the
following schema so we know it will create some range tombstone markers for
every insert of the collection cells.
{noformat}
CREATE TABLE amazon.metadata (
asin text PRIMARY KEY,
also_bought set<text>,
buy_after_viewing set<text>,
categories set<text>,
imurl text,
price double,
title text
);
{noformat}
Once I got the amount of SSTable grew to 7000+, I started JFR to capture the
JVM recording and start the compaction by running "nodetool
enableautocompaction". By analyzing the JFR recording I was able to find
getMaxPurgeableTimestamp() call among the hottest methods for the compaction
thread.
What's interesting is that I also noticed another compaction eating a full CPU
core even after the LCS compaction on amazon.metadata table finished. Turns out
system.sstable_activity is also going through a lot of compaction activities on
its own because it is supposed to be flushed every time a compaction finishes,
so it too generated a lot of tiny SSTables (after enableautocompaction gets
called). Even though system.sstable_activity is using STCS, it still suffered
from the same expensive getMaxPurgeableTimestamp() function call and took a
while to finish. This seems to indicate the problem discussed in this JIRA is
not LCS only, but generally applicable to situations where you have thousands
of tiny SSTables.
IMHO this exercise also prompts us that we may want to consider adding some
dtest to cover the scenario when we have thousands of tiny SSTables, no matter
LCS or STCS is used. This can also be caused by streaming activities like
repair or bootstrapping.
> Ability to disable purgeable tombstone check via startup flag
> -------------------------------------------------------------
>
> Key: CASSANDRA-11831
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11831
> Project: Cassandra
> Issue Type: New Feature
> Reporter: Ryan Svihla
> Assignee: Marcus Eriksson
> Fix For: 2.1.x, 2.2.x, 3.0.x, 3.x
>
>
> On Cassandra 2.1.14 hen a node gets way behind and has 10s of thousand
> sstables it appears a lot of the CPU time is spent doing checks like this on
> a call to getMaxPurgeableTimestamp
> org.apache.cassandra.utils.Murmur3BloomFilter.hash(java.nio.ByteBuffer,
> int, int, long, long[]) @bci=13, line=57 (Compiled frame; information may be
> imprecise)
> - org.apache.cassandra.utils.BloomFilter.indexes(java.nio.ByteBuffer)
> @bci=22, line=82 (Compiled frame)
> - org.apache.cassandra.utils.BloomFilter.isPresent(java.nio.ByteBuffer)
> @bci=2, line=107 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.CompactionController.maxPurgeableTimestamp(org.apache.cassandra.db.DecoratedKey)
> @bci=89, line=186 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.LazilyCompactedRow.getMaxPurgeableTimestamp()
> @bci=21, line=99 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.LazilyCompactedRow.access$300(org.apache.cassandra.db.compaction.LazilyCompactedRow)
> @bci=1, line=49 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.LazilyCompactedRow$Reducer.getReduced()
> @bci=241, line=296 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.LazilyCompactedRow$Reducer.getReduced()
> @bci=1, line=206 (Compiled frame)
> - org.apache.cassandra.utils.MergeIterator$OneToOne.computeNext()
> @bci=44, line=206 (Compiled frame)
> - com.google.common.collect.AbstractIterator.tryToComputeNext() @bci=9,
> line=143 (Compiled frame)
> - com.google.common.collect.AbstractIterator.hasNext() @bci=61, line=138
> (Compiled frame)
> - com.google.common.collect.Iterators$7.computeNext() @bci=4, line=645
> (Compiled frame)
> - com.google.common.collect.AbstractIterator.tryToComputeNext() @bci=9,
> line=143 (Compiled frame)
> - com.google.common.collect.AbstractIterator.hasNext() @bci=61, line=138
> (Compiled frame)
> -
> org.apache.cassandra.db.ColumnIndex$Builder.buildForCompaction(java.util.Iterator)
> @bci=1, line=166 (Compiled frame)
> - org.apache.cassandra.db.compaction.LazilyCompactedRow.write(long,
> org.apache.cassandra.io.util.DataOutputPlus) @bci=52, line=121 (Compiled
> frame)
> -
> org.apache.cassandra.io.sstable.SSTableWriter.append(org.apache.cassandra.db.compaction.AbstractCompactedRow)
> @bci=18, line=193 (Compiled frame)
> -
> org.apache.cassandra.io.sstable.SSTableRewriter.append(org.apache.cassandra.db.compaction.AbstractCompactedRow)
> @bci=13, line=127 (Compiled frame)
> - org.apache.cassandra.db.compaction.CompactionTask.runMayThrow()
> @bci=666, line=197 (Compiled frame)
> - org.apache.cassandra.utils.WrappedRunnable.run() @bci=1, line=28
> (Compiled frame)
> -
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(org.apache.cassandra.db.compaction.CompactionManager$CompactionExecutorStatsCollector)
> @bci=6, line=73 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(org.apache.cassandra.db.compaction.CompactionManager$CompactionExecutorStatsCollector)
> @bci=2, line=59 (Compiled frame)
> -
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run()
> @bci=125, line=264 (Compiled frame)
> - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511
> (Compiled frame)
> - java.util.concurrent.FutureTask.run() @bci=42, line=266 (Compiled frame)
> -
> java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
> @bci=95, line=1142 (Compiled frame)
> - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617
> (Compiled frame)
> - java.lang.Thread.run() @bci=11, line=745 (Compiled frame)
> If we could at least on startup pass a flag like -DskipTombstonePurgeCheck so
> we could in these particularly bad cases just avoid the calculation and merge
> tables until we have less to worry about then restart the node with that flag
> missing once we're down to a more manageable amount of sstables.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)