[ 
https://issues.apache.org/jira/browse/CASSANDRA-13948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16233631#comment-16233631
 ] 

Paulo Motta commented on CASSANDRA-13948:
-----------------------------------------

bq. I was thinking that with 13215 we will have a central point which will keep 
track of when we need to refresh compaction strategies (we could notify CSM 
once cache in the DiskBoundaryManager has been invalidated for example)

Even with that we will probably need to cache the current boundaries on the 
CSM, to prevent a race where the disk boundaries change in the boundary manager 
and a flush puts an SSTable in the wrong strategy (due to the index having 
changed) before the CSM is notified about the boundary change - quite unlikely 
but still possible.

bq. Could we create a new ticket for that though as it is not really related to 
the problem we are trying to solve here

Actually the issue in the original ticket description only surfaced because the 
compaction strategies were not properly reloaded after the disk boundary 
changes, which is the core issue to solve here, so I will update the ticket 
description to better reflect that.

I added two new dtests to check that the compaction strategies are being 
properly reloaded when the disk boundary changes due to bootstrap, decommission 
and delayed join 
([here|https://github.com/pauloricardomg/cassandra-dtest/commit/d53f73419e68eb6925b5baf06824b80d0ccf30b7])
 and was able to reproduce the deadlock in current 3.11/trunk.

While debugging these dtests I found that reloading the compaction strategy 
manager when receiving a notification from the tracker can cause an SSTable to 
be added twice to the {{LeveledManifest}} (first during re-initialization of 
the CS and second when processing the SSTableAddedNotification), so I stopped 
reloading the CSM when receiving a notification from the tracker and added a 
warning on {{LeveledManifest}} when trying to add an SSTable which is already 
present 
([here|https://github.com/pauloricardomg/cassandra/commit/a3eaa8a408cf0e8c5524062fa0fdee9a1eb0d6c0])
 - this shouldn't be a problem since we maybeReload the CSM before submitting a 
new background tasks.

In summary this patch makes the following changes:
1) Reload compaction strategies when JBOD disk boundary changes 
([commit|https://github.com/pauloricardomg/cassandra/commit/efb2afb22792a06d83020ac7097154593b9e684d])
2) Ensure compaction strategies do not loop indefinitely when not able to 
acquire Tracker lock 
([commit|https://github.com/pauloricardomg/cassandra/commit/9fdb8f0fb40954a8ed9570cb568a7084de4c80c5])
3) Only enable compaction strategies after gossip settles to prevent 
unnecessary relocation work 
([commit|https://github.com/pauloricardomg/cassandra/commit/c524ff724f2ca9e7eed59cb07f81b9211098fb5c])
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/a3eaa8a408cf0e8c5524062fa0fdee9a1eb0d6c0])

The CI of the previous version of this patch was successful, so this is ready 
for review. I submitted another round on 3.11 and trunk with the latest version 
and will update the results here when ready.

* [3.11 patch|https://github.com/pauloricardomg/cassandra/tree/3.11-13948]
* [trunk patch|https://github.com/pauloricardomg/cassandra/tree/trunk-13948]

Please let me know what do you think.

> Avoid deadlock when not able to acquire references for compaction
> -----------------------------------------------------------------
>
>                 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
>            Priority: Major
>             Fix For: 3.11.x, 4.x
>
>         Attachments: debug.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

Reply via email to