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

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

There were test failures on:
* testall: {{CompactionsCQLTest.testSetLocalCompactionStrategy}} and 
{{testTriggerMinorCompactionSTCSNodetoolEnabled}}
* dtest: {{disk_balance_test.TestDiskBalance.disk_balance_bootstrap_test}}

{{testSetLocalCompactionStrategy}} and 
{{testTriggerMinorCompactionSTCSNodetoolEnabled}} were failing because when the 
strategy was updated via JMX, these manually set configurations were not 
surviving the compaction strategy reload - this was not introduced by this, but 
would also happen in case a directory was blacklisted before. This was fixed 
[on this 
commit|https://github.com/pauloricardomg/cassandra/commit/11c9a130d9cb7a6cfc5a039fdf79963f7e779d08]

While investigating why the strategies were reloaded even without a ring change 
on the tests above, I noticed that {{Keyspace.createReplicationStrategy}} was 
being called multiple times (on {{Keyspace}} construction and {{setMetadata}}), 
so I updated to only invalidate the disk boundaries when the replication 
settings actually change 
([here|https://github.com/pauloricardomg/cassandra/commit/8a398a5d0d261178547946ac4e457f9abeb90f18]).

After the fix above, {{disk_balance_bootstrap_test}} started failing with 
imbalanced disks because the disk boundaries were not being invalidated after 
the joining node broadcasted its tokens via gossip, so 
{{TokenMetadata.getPendingRanges(keyspace, FBUtilities.getBroadcastAddress())}} 
was returning empty during disk boundary creation and causing imbalance. This 
is not failing on trunk because the double invalidation above during keyspace 
creation was causing the compaction strategy manager to reload the strategies 
with the correct ring placement during streaming. The fix to this is to 
invalidate the cached ring after gossiping the local tokens 
([here|https://github.com/pauloricardomg/cassandra/commit/007d596ffe0c5f965cf398646c52daa8f73c5c46]).

This made me realize that when replacing a node with the same address, even 
though the node is on bootstrap mode, it doesn't have any pending range, 
because it sets its token to normal state during bootstrap, what will cause its 
boundaries to not be computed correctly. I added a 
[dtest|https://github.com/pauloricardomg/cassandra-dtest/commit/8d48b166c9bfce51f9ab6c3abd73dfd4779a7c04]
 to show this and a 
[fix|https://github.com/pauloricardomg/cassandra/commit/6efd9cd454ce2fbfd40e592b6aaeda9debdb1c2b].

Finally, I didn't find a good reason to pass {{ColumnFamilyStore}} as argument 
to {{getDiskBoundaries}}, so I updated it to make it a field instead 
([here|https://github.com/pauloricardomg/cassandra/commit/5df0d5ebed67aaae6ef9350d25b602af2a1702cf]).

I submitted internal CI, and testall is green and dtest failures [seem 
unrelated|https://issues.apache.org/jira/secure/attachment/12899922/dtest2.png].
 Setting to patch available as this should be ready for a new round of review 
now. Thanks!

> 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, dtest13948.png, dtest2.png, 
> threaddump-cleanup.txt, 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

Reply via email to