Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Duplicate
-
None
-
None
-
None
-
Normal
Description
The symptom is similar to what happened in CASSANDRA-13204 that the thread waiting forever doing nothing. This one happened during "nodetool repair -pr -seq -j 1" in production but I can easily simulate the problem with just "nodetool repair" in dev environment (CCM). I'm trying to explain what happened with 3.0.13 code base.
1. One node is down while doing repair. This is the error I saw in production:
ERROR [GossipTasks:1] 2017-05-19 15:00:10,545 RepairSession.java:334 - [repair #bc9a3cd1-3ca3-11e7-a44a-e30923ac9336] session completed with the following error java.io.IOException: Endpoint /10.185.43.15 died at org.apache.cassandra.repair.RepairSession.convict(RepairSession.java:333) ~[apache-cassandra-3.0.11.jar:3.0.11] at org.apache.cassandra.gms.FailureDetector.interpret(FailureDetector.java:306) [apache-cassandra-3.0.11.jar:3.0.11] at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:766) [apache-cassandra-3.0.11.jar:3.0.11] at org.apache.cassandra.gms.Gossiper.access$800(Gossiper.java:66) [apache-cassandra-3.0.11.jar:3.0.11] at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:181) [apache-cassandra-3.0.11.jar:3.0.11] at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) [apache-cassandra-3.0.11.jar:3.0.11] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_121] at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_121] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_121] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_121] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_121] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_121] at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79) [apache-cassandra-3.0.11.jar:3.0.11] at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_121]
2. At this moment the repair coordinator hasn't received the response (MerkleTrees) for the node that was marked down. This means, RepairJob#run will never return because it waits for validations to finish:
// Wait for validation to complete
Futures.getUnchecked(validations);
Be noted that all RepairJob's (as Runnable) run on a shared executor created in RepairRunnable#runMayThrow, while all snapshot, validation and sync'ing happen on a per-RepairSession "taskExecutor". The RepairJob#run will only return when it receives MerkleTrees (or null) from all endpoints for a given column family and token range.
As evidence of the thread leak, below is from the thread dump. I can also get the same stack trace when simulating the same issue in dev environment.
"Repair#129:56" #406373 daemon prio=5 os_prio=0 tid=0x00007fc495028400 nid=0x1a77d waiting on condition [0x00007fc021530000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000002d7c00198> (a com.google.common.util.concurrent.AbstractFuture$Sync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285) at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) at com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:137) at com.google.common.util.concurrent.Futures.getUnchecked(Futures.java:1509) at org.apache.cassandra.repair.RepairJob.run(RepairJob.java:160) 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$4/725832346.run(Unknown Source) at java.lang.Thread.run(Thread.java:745) Locked ownable synchronizers: - <0x00000002d7c00230> (a java.util.concurrent.ThreadPoolExecutor$Worker)
So here are two things:
1. For the thread leak itself, either we do something like below in RepairSession#terminate, or we use timed wait at the end of RepairJob#run.
for (ValidationTask validationTask : validating.values()) { validationTask.treesReceived(null); } validating.clear();
2. Another question is, instead of waiting for synchronization (SyncTask) to finish, why we just wait for validation? Is it because we want to speed things up and anyway we have throttling on streaming?
yukim I'd love to get your comment. I'll check if this issue exists in other versions.
Attachments
Issue Links
- duplicates
-
CASSANDRA-13797 RepairJob blocks on syncTasks
-
- Resolved
-
- relates to
-
CASSANDRA-15902 OOM because repair session thread not closed when terminating repair
-
- Resolved
-