[ 
https://issues.apache.org/jira/browse/RATIS-2240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17913922#comment-17913922
 ] 

Song Ziyang commented on RATIS-2240:
------------------------------------

Simply change `whenCompelete` to `whenCompleteAsync` and let forkjoinpool to 
handle the `updatePurgeIndex` works, but it may introduce extra thread costs. 
What do you think [~szetszwo] (?)

> DeadLock on RaftLogBase.writeLock
> ---------------------------------
>
>                 Key: RATIS-2240
>                 URL: https://issues.apache.org/jira/browse/RATIS-2240
>             Project: Ratis
>          Issue Type: Bug
>          Components: server
>    Affects Versions: 3.1.2
>            Reporter: Song Ziyang
>            Assignee: Song Ziyang
>            Priority: Major
>         Attachments: image-2025-01-17-10-07-04-289.png, 
> image-2025-01-17-10-36-11-163.png, 
> img_v3_02ig_fba1a744-481d-48a1-9bd1-6c1a2784e63g.jpg
>
>
> Deadlock condition observed to block the progress of leader AppendEntries 
> progress.
> !image-2025-01-17-10-36-11-163.png!
> h2. 2-server-thread2 (Server thread handling AppendEntries from leader)
>  * holds lock `RaftLogBase.writeLock`
>  * waits for condition `SegmentedRaftLogWorker.queue.offer`. The queue is 
> full.
> See the thread stack:
>  
> {code:java}
> "2-server-thread2" #152 prio=5 os_prio=0 cpu=535.63ms elapsed=33207.39s 
> tid=0x00007fbb1c00a740 nid=0x42d9 waiting on condition  [0x00007fbaa72f5000]
>    java.lang.Thread.State: TIMED_WAITING (parking)
>     at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
>     - parking to wait for  <0x00007fbe273474f8> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>     at 
> java.util.concurrent.locks.LockSupport.parkNanos([email protected]/LockSupport.java:252)
>     at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos([email protected]/AbstractQueuedSynchronizer.java:1672)
>     at 
> org.apache.ratis.util.DataBlockingQueue.offer(DataBlockingQueue.java:105)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.addIOTask(SegmentedRaftLogWorker.java:278)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.writeLogEntry(SegmentedRaftLogWorker.java:450)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.appendEntryImpl(SegmentedRaftLog.java:428)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.lambda$appendEntry$10(RaftLogBase.java:352)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase$$Lambda$1438/0x00007fbb6c6c90c0.get(Unknown
>  Source)
>     at 
> org.apache.ratis.server.raftlog.RaftLogSequentialOps$Runner.runSequentially(RaftLogSequentialOps.java:78)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.appendEntry(RaftLogBase.java:352)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog.appendImpl(SegmentedRaftLog.java:477)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.lambda$append$11(RaftLogBase.java:359)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase$$Lambda$1435/0x00007fbb6c6c8000.get(Unknown
>  Source)
>     at 
> org.apache.ratis.server.raftlog.RaftLogSequentialOps$Runner.runSequentially(RaftLogSequentialOps.java:69)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.append(RaftLogBase.java:359)
>     at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1598)
>     at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1487)
>     at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:645)
>     at 
> org.apache.ratis.server.impl.RaftServerProxy$$Lambda$1407/0x00007fbb6c6b6248.get(Unknown
>  Source)
>     at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
>     at 
> org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:907)
>     at 
> org.apache.ratis.server.impl.RaftServerImpl$$Lambda$1408/0x00007fbb6c6b6470.get(Unknown
>  Source)
>     at 
> java.util.concurrent.CompletableFuture$AsyncSupply.run([email protected]/CompletableFuture.java:1768)
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
>     at java.lang.Thread.run([email protected]/Thread.java:842)
>  {code}
>  
>  
> h3. SegmentedRaftLogWorker
>  * waits for lock `RaftLogBase.writeLock`
>  * cannot call `SegmentedRaftLogWorker.queue.poll` to remove elements from 
> queue.
> See the thread stack:
>  
> {code:java}
> "2@group-000100000004-SegmentedRaftLogWorker" #90 prio=5 os_prio=0 
> cpu=78140.67ms elapsed=33280.62s tid=0x00007fbafc05f670 nid=0x425b waiting on 
> condition  [0x00007fbb697fe000]
>    java.lang.Thread.State: WAITING (parking)
>     at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
>     - parking to wait for  <0x00007fbe291a3000> (a 
> java.util.concurrent.locks.ReentrantReadWriteLock$FairSync)
>     at 
> java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:211)
>     at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:715)
>     at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire([email protected]/AbstractQueuedSynchronizer.java:938)
>     at 
> java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock([email protected]/ReentrantReadWriteLock.java:959)
>     at 
> org.apache.ratis.util.AutoCloseableLock.acquire(AutoCloseableLock.java:44)
>     at 
> org.apache.ratis.util.AutoCloseableLock.acquire(AutoCloseableLock.java:39)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.writeLock(RaftLogBase.java:375)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.updatePurgeIndex(RaftLogBase.java:141)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase.lambda$purge$9(RaftLogBase.java:336)
>     at 
> org.apache.ratis.server.raftlog.RaftLogBase$$Lambda$1776/0x00007fbb6c83cc98.accept(Unknown
>  Source)
>     at 
> java.util.concurrent.CompletableFuture.uniWhenComplete([email protected]/CompletableFuture.java:863)
>     at 
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire([email protected]/CompletableFuture.java:841)
>     at 
> java.util.concurrent.CompletableFuture.postComplete([email protected]/CompletableFuture.java:510)
>     at 
> java.util.concurrent.CompletableFuture.complete([email protected]/CompletableFuture.java:2147)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog$Task.completeFuture(SegmentedRaftLog.java:103)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLog$Task.done(SegmentedRaftLog.java:99)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker.run(SegmentedRaftLogWorker.java:333)
>     at 
> org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker$$Lambda$999/0x00007fbb6c4b2968.run(Unknown
>  Source)
>     at 
> java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:539)
>     at 
> java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:264)
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635)
>     at java.lang.Thread.run([email protected]/Thread.java:842) {code}
>  
>  * `purge` in `RaftLogBase` (line 319) calls `purgeImpl`, which will be 
> delegated to `SegmentedRaftLog.purgeImpl`.
>  * `SegmentedRaftLog.purgeImpl` will submit a task to 
> `SegmentedRaftLogWorker` and return a future.
>  * The `whenComplete` clause in `purge` in `RaftLogBase` (line 338) will call 
> `updatePurgeIndex` which waits for lock `RaftLogBase.writeLock`. Due to 
> nature of `whenComplete`, this will be executed on thread 
> `SegmentedRaftLogWorker`.
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to