cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Paulo Motta (JIRA)" <j...@apache.org>
Subject [jira] [Created] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction
Date Wed, 11 Oct 2017 06:52:00 GMT
Paulo Motta created CASSANDRA-13948:
---------------------------------------

             Summary: 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()
@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}

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-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org


Mime
View raw message