[ https://issues.apache.org/jira/browse/CASSANDRA-13948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16241711#comment-16241711 ]
Paulo Motta commented on CASSANDRA-13948: ----------------------------------------- bq. Ok I was able to reproduce it on one node. I've attached the trace log. It's unfiltered since I didn't managed to filter only to org.apache.cassandra.db.compaction I wasn't able to track down the root cause of this condition from the logs, but a similar issue was reported on CASSANDRA-12743, so I think this is some kind of race condition showing up due to the amount of concurrent compactions happening and is not a consequence of this fix, so I prefer to investigate this separately. If you still see this issue please feel free to reopen CASSANDRA-12743 with details. bq. However I'm still facing issues with compactions. These are big nodes with with a big CF, holding many SSTables and pending compactions. According the thread dump it seems to be stuck around getNextBackgroundTask. Compactions are still being processed for the other keyspace. Beside that the node is running normally. Some nodetool commands takes time to proceed like compactionstats. Debug log doesn't show any error. After having a look at the thread dump, it turns out that my previous patch generated a lock contention between compaction and cleanup, because each removed SSTable from cleanup generated a {{SSTableDeletingNotification}} and my previous patch submitted a new compaction task after each received notification which competed with the next {{SSTableDeletingNotification}} for the {{writeLock}} - making things slow overall, so I updated the patch to only submit a new compaction after receiving a flush notification as it was before, so this should be fixed now. [~llambiel] would you mind trying the latest version now? [~krummas] this should be ready for review now, the latest version already got a clean CI run, but I resubmitted a new internal CI run after doing the minor fix above and will update here when ready. Summary of changes: 1) Reload compaction strategies when JBOD disk boundary changes ([commit|https://github.com/pauloricardomg/cassandra/commit/6cab7e0a31a638cc4a957c4ecfa592035d874058]) 2) Ensure compaction strategies do not loop indefinitely when not able to acquire Tracker lock ([commit|https://github.com/pauloricardomg/cassandra/commit/3ef833d1e56c25f67bc8a3b49acf97b2efdf401d]) 3) Only enable compaction strategies after gossip settles to prevent unnecessary relocation work ([commit|https://github.com/pauloricardomg/cassandra/commit/eaf63dc3d52566ce0c4f91bbfec478305597f014]) 4) Do not reload compaction strategies when receiving notifications and log warning when an SSTable is added multiple times to LCS ([commit|https://github.com/pauloricardomg/cassandra/commit/3e61df70025e704ee0c9d6ee8754ccdd38f5ab6d]) Patches * [3.11|https://github.com/pauloricardomg/cassandra/tree/3.11-13948] * [trunk|https://github.com/pauloricardomg/cassandra/tree/trunk-13948] I wonder if now that CSM caches the disk boundaries we can make the handling of notifications use the readLock instead of the writeLock, to reduce contention when there is a high number of concurrent compactors, do you see any potential problems with this? Even if the notification handling races with getNextBackground task, as long as the individual compaction strategies are synchronized getNextBackground task should get a consistent view of the strategy sstables when there is a concurrent notification from the tracker. > Reload compaction strategies when JBOD disk boundary changes > ------------------------------------------------------------ > > Key: CASSANDRA-13948 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13948 > Project: Cassandra > Issue Type: Bug > Components: Compaction > Reporter: Paulo Motta > Assignee: Paulo Motta > Fix For: 3.11.x, 4.x > > Attachments: debug.log, threaddump.txt, trace.log > > > The thread dump below shows a race between an sstable replacement by the > {{IndexSummaryRedistribution}} and > {{AbstractCompactionTask.getNextBackgroundTask}}: > {noformat} > Thread 94580: (state = BLOCKED) > - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information > may be imprecise) > - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, > line=175 (Compiled frame) > - > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() > @bci=1, line=836 (Compiled frame) > - > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.util.concurrent.locks.AbstractQueuedSynchronizer$Node, > int) @bci=67, line=870 (Compiled frame) > - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(int) > @bci=17, line=1199 (Compiled frame) > - java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock() @bci=5, > line=943 (Compiled frame) > - > org.apache.cassandra.db.compaction.CompactionStrategyManager.handleListChangedNotification(java.lang.Iterable, > java.lang.Iterable) @bci=359, line=483 (Interpreted frame) > - > org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(org.apache.cassandra.notifications.INotification, > java.lang.Object) @bci=53, line=555 (Interpreted frame) > - > org.apache.cassandra.db.lifecycle.Tracker.notifySSTablesChanged(java.util.Collection, > java.util.Collection, org.apache.cassandra.db.compaction.OperationType, > java.lang.Throwable) @bci=50, line=409 (Interpreted frame) > - > org.apache.cassandra.db.lifecycle.LifecycleTransaction.doCommit(java.lang.Throwable) > @bci=157, line=227 (Interpreted frame) > - > org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit(java.lang.Throwable) > @bci=61, line=116 (Compiled frame) > - > org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit() > @bci=2, line=200 (Interpreted frame) > - > org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish() > @bci=5, line=185 (Interpreted frame) > - > org.apache.cassandra.io.sstable.IndexSummaryRedistribution.redistributeSummaries() > @bci=559, line=130 (Interpreted frame) > - > org.apache.cassandra.db.compaction.CompactionManager.runIndexSummaryRedistribution(org.apache.cassandra.io.sstable.IndexSummaryRedistribution) > @bci=9, line=1420 (Interpreted frame) > - > org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries(org.apache.cassandra.io.sstable.IndexSummaryRedistribution) > @bci=4, line=250 (Interpreted frame) > - > org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries() > @bci=30, line=228 (Interpreted frame) > - org.apache.cassandra.io.sstable.IndexSummaryManager$1.runMayThrow() > @bci=4, line=125 (Interpreted frame) > - org.apache.cassandra.utils.WrappedRunnable.run() @bci=1, line=28 > (Interpreted frame) > - > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run() > @bci=4, line=118 (Compiled frame) > - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 > (Compiled frame) > - java.util.concurrent.FutureTask.runAndReset() @bci=47, line=308 (Compiled > frame) > - > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask) > @bci=1, line=180 (Compiled frame) > - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() > @bci=37, line=294 (Compiled frame) > - > java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) > @bci=95, line=1149 (Compiled frame) > - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 > (Interpreted frame) > - > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable) > @bci=1, line=81 (Interpreted frame) > - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$8.run() @bci=4 > (Interpreted frame) > - java.lang.Thread.run() @bci=11, line=748 (Compiled frame) > {noformat} > {noformat} > Thread 94573: (state = IN_JAVA) > - java.util.HashMap$HashIterator.nextNode() @bci=95, line=1441 (Compiled > frame; information may be imprecise) > - java.util.HashMap$KeyIterator.next() @bci=1, line=1461 (Compiled frame) > - > org.apache.cassandra.db.lifecycle.View$3.apply(org.apache.cassandra.db.lifecycle.View) > @bci=20, line=268 (Compiled frame) > - org.apache.cassandra.db.lifecycle.View$3.apply(java.lang.Object) @bci=5, > line=265 (Compiled frame) > - > org.apache.cassandra.db.lifecycle.Tracker.apply(com.google.common.base.Predicate, > com.google.common.base.Function) @bci=13, line=133 (Compiled frame) > - org.apache.cassandra.db.lifecycle.Tracker.tryModify(java.lang.Iterable, > org.apache.cassandra.db.compaction.OperationType) @bci=31, line=99 (Compiled > frame) > - > org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(int) > @bci=84, line=139 (Compiled frame) > - > org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(int) > @bci=105, line=119 (Interpreted frame) > - > org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run() > @bci=84, line=265 (Interpreted 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=1149 (Compiled frame) > - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 > (Interpreted frame) > - > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable) > @bci=1, line=81 (Interpreted frame) > - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$8.run() @bci=4 > (Interpreted frame) > - java.lang.Thread.run() @bci=11, line=748 (Compiled frame) > {noformat} > This particular node remain in this state forever, indicating > {{LeveledCompactionStrategyTask.getNextBackgroundTask}} was looping > indefinitely. > What happened is that sstable references were replaced on the tracker by the > {{IndexSummaryRedistribution}} thread, so the > {{AbstractCompactionStrategy.getNextBackgroundTask}} could not create the > transaction with the old references, and the {{IndexSummaryRedistribution}} > could not update the sstable reference in the compaction strategy because > {{AbstractCompactionStrategy.getNextBackgroundTask}} was holding the > {{CompactionStrategyManager}} lock. -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org