[
https://issues.apache.org/jira/browse/CASSANDRA-21384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Sam Tunnicliffe updated CASSANDRA-21384:
----------------------------------------
Bug Category: Parent values: Availability(12983)Level 1 values:
Unavailable(12994)
Complexity: Normal
Component/s: Cluster/Gossip
Discovered By: Unit Test
Fix Version/s: 6.x
Severity: Normal
Assignee: Sam Tunnicliffe
Status: Open (was: Triage Needed)
> Deadlock between GlobalLogFollower and GossipStage
> --------------------------------------------------
>
> Key: CASSANDRA-21384
> URL: https://issues.apache.org/jira/browse/CASSANDRA-21384
> Project: Apache Cassandra
> Issue Type: Bug
> Components: Cluster/Gossip, Transactional Cluster Metadata
> Reporter: Dmitry Konstantinov
> Assignee: Sam Tunnicliffe
> Priority: Normal
> Fix For: 6.x
>
> Attachments: stuck_dtest_log.txt.zip
>
>
> Hi, I've noticed DecommissionAvoidReadTimeoutsTest test timeout:
> [https://pre-ci.cassandra.apache.org/job/cassandra/543/pipeline-overview/?selected-node=292]
> Here there is a full log: [^stuck_dtest_log.txt.zip]
> It looks like we have a deadlock here:
> {code:java}
> [2026-05-17T13:39:14.911Z] [junit-timeout] "node3_GlobalLogFollower" #183
> prio=5 os_prio=0 cpu=211.11ms elapsed=890.37s tid=0x00007fe9a423cb30
> nid=0x33b6 waiting on condition [0x00007fe97819c000]
> [2026-05-17T13:39:14.911Z] [junit-timeout] java.lang.Thread.State: WAITING
> (parking)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> jdk.internal.misc.Unsafe.park([email protected]/Native Method)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:341)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AsyncFuture.await(AsyncFuture.java:190)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AsyncFuture.await(AsyncFuture.java:54)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AbstractFuture.get(AbstractFuture.java:249)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.gms.Gossiper.runInGossipStageBlocking(Gossiper.java:544)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.gms.Gossiper.mergeNodeToGossip(Gossiper.java:2263)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.gms.Gossiper.mergeNodeToGossip(Gossiper.java:2248)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.tcm.listeners.LegacyStateListener.processChangesToLocalState(LegacyStateListener.java:160)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.tcm.listeners.LegacyStateListener.notifyPostCommit(LegacyStateListener.java:86)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog.notifyPostCommit(LocalLog.java:618)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:532)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog$Async$AsyncRunnable.run(LocalLog.java:800)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.concurrent.InfiniteLoopExecutor.loop(InfiniteLoopExecutor.java:119)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> org.apache.cassandra.concurrent.InfiniteLoopExecutor$$Lambda$4957/0x00007fea01311a10.run(Unknown
> Source)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> [2026-05-17T13:39:14.911Z] [junit-timeout] at
> java.lang.Thread.run([email protected]/Thread.java:840)
> [2026-05-17T13:39:14.911Z] [junit-timeout]
> [2026-05-17T13:39:14.911Z] [junit-timeout] Locked ownable synchronizers:
> [2026-05-17T13:39:14.911Z] [junit-timeout] - <0x00000000c81936a8> (a
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
> [2026-05-17T13:39:14.911Z] [junit-timeout]
> [2026-05-17T13:39:14.924Z] [junit-timeout] "node3_GossipStage:1" #303 daemon
> prio=5 os_prio=0 cpu=27.30ms elapsed=888.43s tid=0x00007fe9a8019d90
> nid=0x342e waiting on condition [0x00007fe951cee000]
> [2026-05-17T13:39:14.924Z] [junit-timeout] java.lang.Thread.State: WAITING
> (parking)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> jdk.internal.misc.Unsafe.park([email protected]/Native Method)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:341)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:322)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.WaitQueue$Standard$AbstractSignal.await(WaitQueue.java:300)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.Awaitable$AsyncAwaitable.await(Awaitable.java:306)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.Awaitable$AsyncAwaitable.await(Awaitable.java:338)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.Awaitable$Defaults.awaitThrowUncheckedOnInterrupt(Awaitable.java:131)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.Awaitable$AbstractAwaitable.awaitThrowUncheckedOnInterrupt(Awaitable.java:235)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog$Async.runOnce(LocalLog.java:717)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog.runOnce(LocalLog.java:437)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.log.LocalLog.waitForHighestConsecutive(LocalLog.java:356)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher.lambda$fetchLogEntriesAndWaitInternal$1(PeerLogFetcher.java:103)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher$$Lambda$14351/0x00007fea02cfb390.apply(Unknown
> Source)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AbstractFuture.lambda$map$0(AbstractFuture.java:345)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AbstractFuture$$Lambda$12637/0x00007fea029a8e18.run(Unknown
> Source)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.concurrent.ImmediateExecutor.execute(ImmediateExecutor.java:140)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList.safeExecute(ListenerList.java:193)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList.notifyListener(ListenerList.java:184)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList$RunnableWithExecutor.notifySelf(ListenerList.java:372)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList.lambda$notifyExclusive$0(ListenerList.java:151)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList$$Lambda$7194/0x00007fea01ca3890.accept(Unknown
> Source)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.IntrusiveStack$$Lambda$7196/0x00007fea01ca3ac8.accept(Unknown
> Source)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.IntrusiveStack.forEach(IntrusiveStack.java:244)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.IntrusiveStack.forEach(IntrusiveStack.java:237)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.IntrusiveStack.forEach(IntrusiveStack.java:227)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList.notifyExclusive(ListenerList.java:151)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.ListenerList.notify(ListenerList.java:116)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AsyncFuture.appendListener(AsyncFuture.java:122)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AbstractFuture.addListener(AbstractFuture.java:433)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AbstractFuture.map(AbstractFuture.java:342)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.AsyncFuture.map(AsyncFuture.java:144)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.utils.concurrent.Future.map(Future.java:166)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher.fetchLogEntriesAndWaitInternal(PeerLogFetcher.java:101)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher.lambda$asyncFetchLog$0(PeerLogFetcher.java:78)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher$$Lambda$14347/0x00007fea02cfac90.get(Unknown
> Source)
> [2026-05-17T13:39:14.924Z] [junit-timeout] at
> org.apache.cassandra.tcm.EpochAwareDebounce.getAsync(EpochAwareDebounce.java:83)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.tcm.PeerLogFetcher.asyncFetchLog(PeerLogFetcher.java:78)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.tcm.ClusterMetadataService.fetchLogFromPeerAsync(ClusterMetadataService.java:761)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.gms.GossipVerbHandler.doVerb(GossipVerbHandler.java:32)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(GossipDigestAckVerbHandler.java:92)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink.lambda$new$0(InboundSink.java:104)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink$$Lambda$10290/0x00007fea022217a0.accept(Unknown
> Source)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:72)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink$Filtered.accept(InboundSink.java:58)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink.accept(InboundSink.java:124)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundSink.accept(InboundSink.java:53)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.net.InboundMessageHandler$ProcessMessage.run(InboundMessageHandler.java:460)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:138)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> [2026-05-17T13:39:14.925Z] [junit-timeout] at
> java.lang.Thread.run([email protected]/Thread.java:840)
> [2026-05-17T13:39:14.925Z] [junit-timeout]
> [2026-05-17T13:39:14.925Z] [junit-timeout] Locked ownable synchronizers:
> [2026-05-17T13:39:14.925Z] [junit-timeout] - <0x00000000d7f82ff8> (a
> java.util.concurrent.ThreadPoolExecutor$Worker)
> {code}
>
> An analysis from Claude (after few iterations):
> h2. Root Cause: Circular deadlock between TCM log processing and GossipStage
> on node3
> The test never reaches the decommission phase. The cluster hangs during
> startup because
> of a circular wait between node3's TCM (Transactional Cluster Metadata) log
> processor
> thread and its GossipStage thread.
> h3. The deadlock cycle
> {code:java}
> node3_GlobalLogFollower
> → processPendingInternal()
> → notifyPostCommit()
> → LegacyStateListener.processChangesToLocalState()
> → Gossiper.mergeNodeToGossip()
> → Gossiper.runInGossipStageBlocking() ← BLOCKS waiting for GossipStage
> ↑
> │ waits for
> │
> node3_GossipStage:1 │
> → GossipVerbHandler.doVerb() │
> → ClusterMetadataService.fetchLogFromPeerAsync() │
> → PeerLogFetcher.fetchLogEntriesAndWaitInternal() │
> → LocalLog.waitForHighestConsecutive() │
> → LocalLog.runOnce() ← BLOCKS waiting for log to advance
> │
> can't advance because
> GlobalLogFollower is stuck ↑
> {code}
> h3. Why this deadlocks
> 1. *node3_GossipStage* receives a gossip message
> ({{{}GossipVerbHandler.doVerb{}}}), discovers the peer has a newer TCM epoch,
> and calls {{{}fetchLogFromPeerAsync(){}}}. This triggers a synchronous wait
> ({{{}waitForHighestConsecutive{}}}) for the local log to catch up to the
> fetched epoch. It *occupies the single GossipStage thread* while waiting.
> 2. *node3_GlobalLogFollower* is the thread that processes TCM log entries.
> When processing a new entry, it fires post-commit listeners. The
> {{LegacyStateListener}} calls
> {{Gossiper.runInGossipStageBlocking()}} which submits work to the GossipStage
> and blocks waiting for it to complete.
> 3. Since GossipStage is already occupied by step 1, GlobalLogFollower blocks.
> Since
> GlobalLogFollower is blocked, the local log never advances. Since the log
> never advances,
> GossipStage's wait never completes. *Deadlock.*
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]