[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-31 Thread Paulo Motta (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-13948?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
>  - 
> 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-19 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13948:
-

bq. Also this is a bit orthogonal to CASSANDRA-13215
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)

bq. So it shouldn't be a problem if on a race with gossip flush writes an 
SSTable to a wrong disk,
The flushed sstable will have wrong boundaries, it will have tokens that 
shouldn't be on that disk, does not really matter if the first token is on the 
correct disk - more tokens might be on the correct disk than not - this is why 
6696 didn't reload compaction strategies after boundary changes. I didn't 
really consider the case from these logs though, where we are completely wrong 
at startup, but then go back to the correct pre-restart state, then it totally 
makes sense to reload them, and I agree, we should always do it for consistency.

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

> 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
> 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)
>  - 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-19 Thread Paulo Motta (JIRA)

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

Paulo Motta commented on CASSANDRA-13948:
-

bq. we should probably fix that instead (or, as well) - flushing sstables also 
depends on the boundaries, and we can't delay that until gossip has settled 
(commitlog replay might have to flush)

The reason why the ring boundaries were not computed correctly on startup was 
because the {{GossipingPropertyFileSnitch}} did not have rack/dc info about all 
nodes on gossip so it fallback to the sample 
{{conf/cassandra-rackdc.properties}} file from {{PropertyFileSnitch}} so I 
created CASSANDRA-13970 to fix that.

In any case, even with that fixed, the {{CompactionStrategyManager}} (CSM) does 
not reload its compaction strategies when the disk boundaries are updated 
(either because of range movements, when the node first joins the ring, or a 
disk breaks) what can cause 
{{CompactionStrategyManager.getCompactionStrategyIndex}} to return a different 
sstable->disk assignment to the one currently on the CSM, so some SStables may 
not be correctly updated in the compaction strategies after being 
updated/replaced/removed.  Perhaps this is better illustrated by [this 
test|https://github.com/pauloricardomg/cassandra/commit/073bfa4e548ac807b523a919288a5a71379bfd21#diff-f9c882c974db60a710cf1f195cfdb801R95].

So it shouldn't be a problem if on a race with gossip flush writes an SSTable 
to a wrong disk, as long as after the boundary is updated, the SSTable is 
placed on the correct compaction strategy and its compaction output will be 
placed in the correct disk (should probably add a test with this scenario).

Also this is a bit orthogonal to CASSANDRA-13215 - the objective there is to 
cache the disk boundary computation, while here is to make sure CSM can 
gracefully handle boundary changes.

> 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
> 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)
>  - 
> 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-19 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13948:
-

bq. It turns out that the ColumnFamilyStore (and subsequently the 
CompactionStrategyManager) is initialized before gossip is settled, so the 
node's local ranges are not properly computed during startup, causing the 
computed disk boundaries to not match the actual boundaries.
we should probably fix that instead (or, as well) - flushing sstables also 
depends on the boundaries, and we can't delay that until gossip has settled 
(commitlog replay might have to flush)

> 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
> 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() 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-19 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13948:
-

have not had a close look yet, but did you see CASSANDRA-13215 ?

> 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
> 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 = 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-19 Thread Paulo Motta (JIRA)

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

Paulo Motta commented on CASSANDRA-13948:
-

I think I was able to get to the bottom of this issue with the help of 
[~dkinder]'s logs

It turns out that the {{ColumnFamilyStore}} (and subsequently the 
{{CompactionStrategyManager}}) is initialized before gossip is settled, so the 
node's local ranges are not properly computed during startup, causing the 
computed disk boundaries to not match the actual boundaries. 

This happens because {{GossipingPropertyFileSnitch}} fallbacks to the 
{{FilePropertySnitch}}, so when a node is not found is gossip it will pick the 
DCs/racks from the {{cassandra-topology.properties}} file, and if it's not 
defined there it will use the {{DEFAULT}} dc/rack and mess up the local ranges 
and disk boundary computation.

The log lines below show the following steps:
* {{GossipingPropertyFileSnitch}} falling back to {{PropertyFileSnitch}}
* Compactions on system keyspaces being scheduled on the same disk, while 
compaction on user keyspaces being run on SSTables from different disks, 
indicating the disk boundaries were not calculated correctly for NTS keyspaces
* After gossip settles, lot's of {{SSTable from level 0 is not on corresponding 
level in the leveled manifest}} warnings, indicating the disk boundary layout 
changed after gossip settled but the compaction strategies were not reloaded 
with the new layout

{code:none}
INFO  [main] 2017-10-12 14:39:32,928 GossipingPropertyFileSnitch.java:64 - 
Loaded cassandra-topology.properties for compatibility
DEBUG [CompactionExecutor:26] 2017-10-12 15:02:09,442 CompactionTask.java:155 - 
Compacting (fe490ea1-af98-11e7-b5a1-57bcefdac924) 
[/srv/disk6/cassandra-data/walker/domain_info/.domain_info_claim_tok_idx/mc-18386-big-Data.db:level=0,
 
/srv/disk9/cassandra-data/walker/domain_info/.domain_info_claim_tok_idx/mc-18387-big-Data.db:level=0,
 ]
DEBUG [CompactionExecutor:31] 2017-10-12 15:02:09,442 CompactionTask.java:155 - 
Compacting (fe490ea0-af98-11e7-b5a1-57bcefdac924) 
[/srv/disk10/cassandra-data/system/peers/mc-1671-big-Data.db:level=0, 
/srv/disk10/cassandra-data/system/peers/mc-1659-big-Data.db:level=0, 
/srv/disk10/cassandra-data/system/peers/mc-1656-big-Data.db:level=0, 
/srv/disk10/cassandra-data/system/peers/mc-1690-big-Data.db:level=0, ]
DEBUG [CompactionExecutor:17] 2017-10-12 15:02:09,442 CompactionTask.java:155 - 
Compacting (fe490ea8-af98-11e7-b5a1-57bcefdac924) 
[/srv/disk5/cassandra-data/system_schema/columns-24101c25a2ae3af787c1b40ee1aca33f/mc-2625-big-Data.db:level=0,
 
/srv/disk5/cassandra-data/system_schema/columns-24101c25a2ae3af787c1b40ee1aca33f/mc-2605-big-Data.db:level=0,
 
/srv/disk5/cassandra-data/system_schema/columns-24101c25a2ae3af787c1b40ee1aca33f/mc-2597-big-Data.db:level=0,
 
/srv/disk5/cassandra-data/system_schema/columns-24101c25a2ae3af787c1b40ee1aca33f/mc-2617-big-Data.db:level=0,
 ]
DEBUG [CompactionExecutor:29] 2017-10-12 15:02:09,952 CompactionTask.java:155 - 
Compacting (fe9a8a00-af98-11e7-b5a1-57bcefdac924) 
[/srv/disk11/cassandra-data/walker/domain_info/.domain_info_dispatched_idx/mc-18474-big-Data.db:level=0,
 
/srv/disk3/cassandra-data/walker/domain_info/.domain_info_dispatched_idx/mc-18473-big-Data.db:level=0,
 ]
DEBUG [CompactionExecutor:18] 2017-10-12 15:04:08,939 CompactionTask.java:155 - 
Compacting (45865ca0-af99-11e7-b5a1-57bcefdac924) 
[/srv/disk8/cassandra-data/walker/links/mc-6571323-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566335-big-Data.db:level=1, 
/srv/disk7/cassandra-data/walker/links/mc-6566315-big-Data.db:level=1, 
/srv/disk7/cassandra-data/walker/links/mc-6566361-big-Data.db:level=1, 
/srv/disk8/cassandra-data/walker/links/mc-6571043-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566330-big-Data.db:level=1, 
/srv/disk7/cassandra-data/walker/links/mc-6566322-big-Data.db:level=1, 
/srv/disk8/cassandra-data/walker/links/mc-6571261-big-Data.db:level=0, 
/srv/disk8/cassandra-data/walker/links/mc-6571335-big-Data.db:level=0, 
/srv/disk8/cassandra-data/walker/links/mc-6570145-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566346-big-Data.db:level=1, 
/srv/disk8/cassandra-data/walker/links/mc-6571249-big-Data.db:level=0, 
/srv/disk8/cassandra-data/walker/links/mc-6571311-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566307-big-Data.db:level=1, 
/srv/disk8/cassandra-data/walker/links/mc-6571285-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566341-big-Data.db:level=1, 
/srv/disk8/cassandra-data/walker/links/mc-6570765-big-Data.db:level=0, 
/srv/disk8/cassandra-data/walker/links/mc-6571299-big-Data.db:level=0, 
/srv/disk7/cassandra-data/walker/links/mc-6566352-big-Data.db:level=1, ]
DEBUG [CompactionExecutor:30] 2017-10-12 15:04:08,866 
DiskBoundaryManager.java:69 - Cached ring 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-17 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13948:
-

[~pauloricardomg] did you get those CI results?

> 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
> 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)
>  - 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder commented on CASSANDRA-13948:


Just a heads up, I have been seeing these deadlocks happen easily, so I am 
running your patch [~pauloricardomg] in addition to 
Marcus's[patch|https://github.com/krummas/cassandra/commits/marcuse/13215] from 
CASSANDRA-13215.

I do see a large number of "Could not acquire references for compacting 
SSTables 
[BigTableReader(path='/srv/disk2/cassandra-data/walker/links/mc-6566879-big-Data.db')]
 which is not a problem per se,unless it happens frequently, in which case it 
must be reported. Will retry later." happening, in bursts. Will upload a log 
file.

I also see some of this:
{noformat}
java.lang.AssertionError: Memory was freed
at org.apache.cassandra.io.util.Memory.checkBounds(Memory.java:344) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at org.apache.cassandra.io.util.Memory.getInt(Memory.java:291) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:148)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.fillTemporaryKey(IndexSummary.java:162)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:1370)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.estimatedKeysForRanges(SSTableReader.java:1326)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:441)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:503)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:124)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:262)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_144]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[na:1.8.0_144]
at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
 [apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_144]
{noformat}

> 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
> Fix For: 3.11.x, 4.x
>
>
> 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 

[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-11 Thread Paulo Motta (JIRA)

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

Paulo Motta commented on CASSANDRA-13948:
-

I think we can get rid of the {{while (true)}} loops on 
{{*CompactionStrategy.getNextBackgroundTask}} when not able to lock sstables 
for compaction, and just submit a new background task when receiving any 
notifications from the tracker to ensure a new compaction will operate on the 
most updated references.

I also removed the {{CompactionStrategyManager.replaceFlushed}} method because 
it should no longer be necessary because a new background compaction candidate 
will be submitted when receiving an {{SSTableAddedNotification}} from the 
tracker.

I added a unit test to check that {{*CompactionStrategy.getNextBackgroundTask}} 
never blocks indefinitely, even when not able to lock sstables in the tracker.

A race there should be pretty unlikely, but in case it happens I logged a 
warning to detect potential problems if it happens frequently due to some wrong 
condition: {{"Could not acquire references for compacting SSTables {} which is 
not a problem per se, unless it happens frequently, in which case it must be 
reported. Will retry later."}}.

Patch available 
[here|https://github.com/pauloricardomg/cassandra/tree/3.11-13948]

Mind having a look [~krummas]?

I submitted internal CI, will post the results here once available.

> 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
> Fix For: 3.11.x, 4.x
>
>
> 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()
>