[
https://issues.apache.org/jira/browse/CASSANDRA-13652?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16082776#comment-16082776
]
Ariel Weisberg commented on CASSANDRA-13652:
--------------------------------------------
Although TBH thinking on it why tempt fate with LockSupport.unpark without
checking the thread is actually blocked on what we think it is? Let's go with
the semaphore and drop the wait at line 130.
> Deadlock in AbstractCommitLogSegmentManager
> -------------------------------------------
>
> Key: CASSANDRA-13652
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13652
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Fuud
>
> AbstractCommitLogManager uses LockSupport.(un)park incorreclty. It invokes
> unpark without checking if manager thread was parked in approriate place.
> For example, logging frameworks uses queues and queues uses ReadWriteLock's
> that uses LockSupport. Therefore AbstractCommitLogManager.wakeManager can
> wake thread inside Lock and manager thread will sleep forever at park()
> method (because unpark permit was already consumed inside lock).
> For examle stack traces:
> {code}
> "MigrationStage:1" id=412 state=WAITING
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> at
> org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.awaitAvailableSegment(AbstractCommitLogSegmentManager.java:263)
> at
> org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.advanceAllocatingFrom(AbstractCommitLogSegmentManager.java:237)
> at
> org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager.forceRecycleAll(AbstractCommitLogSegmentManager.java:279)
> at
> org.apache.cassandra.db.commitlog.CommitLog.forceRecycleAllSegments(CommitLog.java:210)
> at org.apache.cassandra.config.Schema.dropView(Schema.java:708)
> at
> org.apache.cassandra.schema.SchemaKeyspace.lambda$updateKeyspace$23(SchemaKeyspace.java:1361)
> at
> org.apache.cassandra.schema.SchemaKeyspace$$Lambda$382/1123232162.accept(Unknown
> Source)
> at java.util.LinkedHashMap$LinkedValues.forEach(LinkedHashMap.java:608)
> at
> java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080)
> at
> org.apache.cassandra.schema.SchemaKeyspace.updateKeyspace(SchemaKeyspace.java:1361)
> at
> org.apache.cassandra.schema.SchemaKeyspace.mergeSchema(SchemaKeyspace.java:1332)
> at
> org.apache.cassandra.schema.SchemaKeyspace.mergeSchemaAndAnnounceVersion(SchemaKeyspace.java:1282)
> - locked java.lang.Class@cc38904
> at
> org.apache.cassandra.db.DefinitionsUpdateVerbHandler$1.runMayThrow(DefinitionsUpdateVerbHandler.java:51)
> at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$LocalSessionWrapper.run(DebuggableThreadPoolExecutor.java:322)
> at
> com.ringcentral.concurrent.executors.MonitoredRunnable.run(MonitoredRunnable.java:36)
> at MON_R_MigrationStage.run(NamedRunnableFactory.java:67)
> at
> com.ringcentral.concurrent.executors.MonitoredThreadPoolExecutor$MdcAwareRunnable.run(MonitoredThreadPoolExecutor.java:114)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/1733339045.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:745)
> "COMMIT-LOG-ALLOCATOR:1" id=80 state=WAITING
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at
> org.apache.cassandra.db.commitlog.AbstractCommitLogSegmentManager$1.runMayThrow(AbstractCommitLogSegmentManager.java:128)
> at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$61/1733339045.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> Solution is to use Semaphore instead of low-level LockSupport.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]