[
https://issues.apache.org/jira/browse/CASSANDRA-13849?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Josh McKenzie updated CASSANDRA-13849:
--------------------------------------
Bug Category: Parent values: Availability(12983)Level 1 values:
Unavailable(12994)
> GossipStage blocks because of race in ActiveRepairService
> ---------------------------------------------------------
>
> Key: CASSANDRA-13849
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13849
> Project: Cassandra
> Issue Type: Bug
> Reporter: Tom van der Woerdt
> Assignee: Sergey Lapukhov
> Priority: Normal
> Labels: patch
> Fix For: 3.0.16, 3.11.2, 4.0
>
> Attachments: CAS-13849.patch, CAS-13849_2.patch, CAS-13849_3.patch
>
>
> Bad luck caused a kernel panic in a cluster, and that took another node with
> it because GossipStage stopped responding.
> I think it's pretty obvious what's happening, here are the relevant excerpts
> from the stack traces :
> {noformat}
> "Thread-24004" #393781 daemon prio=5 os_prio=0 tid=0x00007efca9647400
> nid=0xe75c waiting on condition [0x00007efaa47fe000]
> java.lang.Thread.State: TIMED_WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x000000052b63a7e8> (a
> java.util.concurrent.CountDownLatch$Sync)
> at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
> at
> org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:332)
> - locked <0x00000002e6bc99f0> (a
> org.apache.cassandra.service.ActiveRepairService)
> at
> org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:211)
> 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.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:748)
> "GossipTasks:1" #367 daemon prio=5 os_prio=0 tid=0x00007efc5e971000
> nid=0x700b waiting for monitor entry [0x00007dfb839fe000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at
> org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421)
> - waiting to lock <0x00000002e6bc99f0> (a
> org.apache.cassandra.service.ActiveRepairService)
> at
> org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776)
> at
> org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306)
> at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:775)
>
> at
> org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:67)
> at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:187)
> at
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:748)
> "GossipStage:1" #320 daemon prio=5 os_prio=0 tid=0x00007efc5b9f2c00
> nid=0x6fcd waiting for monitor entry [0x00007e260186a000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at
> org.apache.cassandra.service.ActiveRepairService.removeParentRepairSession(ActiveRepairService.java:421)
> - waiting to lock <0x00000002e6bc99f0> (a
> org.apache.cassandra.service.ActiveRepairService)
> at
> org.apache.cassandra.service.ActiveRepairService.convict(ActiveRepairService.java:776)
> at
> org.apache.cassandra.service.ActiveRepairService.onRestart(ActiveRepairService.java:744)
> at
> org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:1049)
> at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1143)
> at
> org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:49)
> at
> org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:67)
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
> at
> org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$3/1498438472.run(Unknown
> Source)
> at java.lang.Thread.run(Thread.java:748)
> {noformat}
> iow, org.apache.cassandra.service.ActiveRepairService.prepareForRepair holds
> a lock until the repair is prepared, which means waiting for other nodes to
> respond, which may die at exactly that moment, so they won't complete. Gossip
> will at the same time try to mark the node as down, but it requires that same
> lock :)
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]