[ 
https://issues.apache.org/jira/browse/RATIS-2116?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Haibo Sun updated RATIS-2116:
-----------------------------
    Description: 
Using version 2.5.1, we have discovered that in some cases, the state 
synchronization of the follower will be permanently blocked.

Scenario: When the task queue of the SegmentedRaftLogWorker is the pattern 
(WriteLog, WriteLog, ..., PurgeLog), the last WriteLog of 
RaftServerImpl.appendEntries does not immediately flush data and complete the 
result future, because there is a pending PurgeLog task in the queue. It 
enqueues the result future to be completed after the latter WriteLog flushes 
data. However, the "nioEventLoopGroup-3-1" thread is already blocked, and will 
not add new WriteLog to the task queue of SegmentedRaftLogWorker. This leads to 
a deadlock and causes the state synchronization to stop.

I confirmed this by adding debug logs, detailed information is attached below. 
This issue can be easily reproduced by increasing the frequency of TakeSnapshot 
and PurgeLog operations. In addition, after checking the code in the master 
branch, this issue still exists.
 
*jstack:*
"nioEventLoopGroup-3-1" #58 prio=10 os_prio=0 tid=0x00007fc58400b800 
nid=0x5493a waiting on condition [0x00007fc5b4f28000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park0(Native Method)
 - parking to wait for <0x00007fd86a4685e8> (a 
java.util.concurrent.CompletableFuture$Signaller)
at sun.misc.Unsafe.park(Unsafe.java:1025)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:176)
at 
java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
at 
java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
at java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1934)
at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntries(RaftServerImpl.java:1379)
at 
org.apache.ratis.server.impl.RaftServerProxy.appendEntries(RaftServerProxy.java:649)
at 
org.apache.ratis.netty.server.NettyRpcService.handle(NettyRpcService.java:231)
at 
org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:95)
at 
org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:91)
at 
org.apache.ratis.thirdparty.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
at 
org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at 
org.apache.ratis.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at 
org.apache.ratis.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:882)

 
*The debug logs which confirmed the cause:* 
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] INFO 
raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: snapshotIndex: 
updateIncreasingly 17818160 -> 17868160
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
raftlog.RaftLog - purging segments:toTruncate: null
toDelete: [(17817216, 17818372) isOpen? false, length=0, newEndIndex=0, 
(17818373, 17819224) isOpen? false, length=0, newEndIndex=0, (17819225, 
17820474) isOpen? false, length=0, newEndIndex=0, (17820475, 17821368) isOpen? 
false, length=0, newEndIndex=0, (17821369, 17822360) isOpen? false, length=0, 
newEndIndex=0, (17822361, 17823347) isOpen? false, length=0, newEndIndex=0, 
(17823348, 17824136) isOpen? false, length=0, newEndIndex=0, (17824137, 
17825440) isOpen? false, length=0, newEndIndex=0, (17825441, 17826153) isOpen? 
false, length=0, newEndIndex=0, (17826154, 17826936) isOpen? false, length=0, 
newEndIndex=0, (17826937, 17827393) isOpen? false, length=0, newEndIndex=0, 
(17827394, 17828569) isOpen? false, length=0, newEndIndex=0, (17828570, 
17829095) isOpen? false, length=0, newEndIndex=0, (17829096, 17830279) isOpen? 
false, length=0, newEndIndex=0, (17830280, 17831209) isOpen? false, length=0, 
newEndIndex=0, (17831210, 17831887) isOpen? false, length=0, newEndIndex=0, 
(17831888, 17832491) isOpen? false, length=0, newEndIndex=0, (17832492, 
17833577) isOpen? false, length=0, newEndIndex=0, (17833578, 17834387) isOpen? 
false, length=0, newEndIndex=0, (17834388, 17835369) isOpen? false, length=0, 
newEndIndex=0, (17835370, 17836107) isOpen? false, length=0, newEndIndex=0, 
(17836108, 17837307) isOpen? false, length=0, newEndIndex=0, (17837308, 
17838140) isOpen? false, length=0, newEndIndex=0, (17838141, 17838591) isOpen? 
false, length=0, newEndIndex=0, (17838592, 17839411) isOpen? false, length=0, 
newEndIndex=0, (17839412, 17840613) isOpen? false, length=0, newEndIndex=0, 
(17840614, 17841672) isOpen? false, length=0, newEndIndex=0, (17841673, 
17842917) isOpen? false, length=0, newEndIndex=0, (17842918, 17843636) isOpen? 
false, length=0, newEndIndex=0, (17843637, 17844560) isOpen? false, length=0, 
newEndIndex=0, (17844561, 17847272) isOpen? false, length=0, newEndIndex=0, 
(17847273, 17847746) isOpen? false, length=0, newEndIndex=0, (17847747, 
17848169) isOpen? false, length=0, newEndIndex=0, (17848170, 17848631) isOpen? 
false, length=0, newEndIndex=0, (17848632, 17849078) isOpen? false, length=0, 
newEndIndex=0, (17849079, 17850026) isOpen? false, length=0, newEndIndex=0, 
(17850027, 17850490) isOpen? false, length=0, newEndIndex=0, (17850491, 
17851382) isOpen? false, length=0, newEndIndex=0, (17851383, 17852289) isOpen? 
false, length=0, newEndIndex=0, (17852290, 17853645) isOpen? false, length=0, 
newEndIndex=0, (17853646, 17854151) isOpen? false, length=0, newEndIndex=0, 
(17854152, 17855586) isOpen? false, length=0, newEndIndex=0, (17855587, 
17856772) isOpen? false, length=0, newEndIndex=0, (17856773, 17857571) isOpen? 
false, length=0, newEndIndex=0, (17857572, 17858043) isOpen? false, length=0, 
newEndIndex=0, (17858044, 17858598) isOpen? false, length=0, newEndIndex=0, 
(17858599, 17859922) isOpen? false, length=0, newEndIndex=0, (17859923, 
17860327) isOpen? false, length=0, newEndIndex=0, (17860328, 17861304) isOpen? 
false, length=0, newEndIndex=0, (17861305, 17862381) isOpen? false, length=0, 
newEndIndex=0, (17862382, 17863572) isOpen? false, length=0, newEndIndex=0, 
(17863573, 17864018) isOpen? false, length=0, newEndIndex=0, (17864019, 
17865114) isOpen? false, length=0, newEndIndex=0, (17865115, 17866739) isOpen? 
false, length=0, newEndIndex=0, (17866740, 17867551) isOpen? false, length=0, 
newEndIndex=0]
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
segmented.SegmentedRaftLogWorker - 1@group-47BEDE733167-SegmentedRaftLogWorker 
adds IO task PurgeLog:17867551
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
segmented.SegmentedRaftLogWorker - added IO task (queueSize: 5) 
PurgeLog:17867551
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
raftlog.RaftLog - complete future of IO task WriteLog:17876749
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876749
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 4) 
WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 3) 
WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 2) 
WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 1) 
WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 0) 
PurgeLog:17867551
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] INFO 
raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: purgeIndex: 
updateToMax old=17817215, new=17867551, updated? true
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
raftlog.RaftLog - complete future of IO task PurgeLog:17867551
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task PurgeLog:17867551

  was:
Using version 2.5.1, we have discovered that in some cases, the state 
synchronization of the follower will be permanently blocked.

Scenario: When the task queue of the SegmentedRaftLogWorker is the pattern 
(WriteLog, WriteLog, ..., PurgeLog), the last WriteLog of 
RaftServerImpl.appendEntries does not immediately flush data and complete the 
result future, because there is a pending PurgeLog task in the queue. It 
enqueues the result future to be completed
after the latter WriteLog flushes data. However, the "nioEventLoopGroup-3-1" 
thread is already blocked, and will not add new WriteLog to the task queue of 
SegmentedRaftLogWorker. This leads to a deadlock and causes the state 
synchronization to stop.

I confirmed this by adding debug logs, detailed information is attached below. 
This issue can be easily reproduced by increasing the frequency of TakeSnapshot 
and PurgeLog operations. In addition, after checking the code in the master 
branch, this issue still exists.
 
*jstack:*
"nioEventLoopGroup-3-1" #58 prio=10 os_prio=0 tid=0x00007fc58400b800 
nid=0x5493a waiting on condition [0x00007fc5b4f28000]
java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park0(Native Method)
- parking to wait for <0x00007fd86a4685e8> (a 
java.util.concurrent.CompletableFuture$Signaller)
at sun.misc.Unsafe.park(Unsafe.java:1025)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:176)
at 
java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
at 
java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
at java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1934)
at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntries(RaftServerImpl.java:1379)
at 
org.apache.ratis.server.impl.RaftServerProxy.appendEntries(RaftServerProxy.java:649)
at 
org.apache.ratis.netty.server.NettyRpcService.handle(NettyRpcService.java:231)
at 
org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:95)
at 
org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:91)
at 
org.apache.ratis.thirdparty.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
at 
org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
at 
org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
at 
org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
at 
org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
at 
org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
at 
org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
at 
org.apache.ratis.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
at 
org.apache.ratis.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(Thread.java:882)


 
*The debug logs which confirmed the cause:* 
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] INFO 
raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: snapshotIndex: 
updateIncreasingly 17818160 -> 17868160
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
raftlog.RaftLog - purging segments:toTruncate: null
toDelete: [(17817216, 17818372) isOpen? false, length=0, newEndIndex=0, 
(17818373, 17819224) isOpen? false, length=0, newEndIndex=0, (17819225, 
17820474) isOpen? false, length=0, newEndIndex=0, (17820475, 17821368) isOpen? 
false, length=0, newEndIndex=0, (17821369, 17822360) isOpen? false, length=0, 
newEndIndex=0, (17822361, 17823347) isOpen? false, length=0, newEndIndex=0, 
(17823348, 17824136) isOpen? false, length=0, newEndIndex=0, (17824137, 
17825440) isOpen? false, length=0, newEndIndex=0, (17825441, 17826153) isOpen? 
false, length=0, newEndIndex=0, (17826154, 17826936) isOpen? false, length=0, 
newEndIndex=0, (17826937, 17827393) isOpen? false, length=0, newEndIndex=0, 
(17827394, 17828569) isOpen? false, length=0, newEndIndex=0, (17828570, 
17829095) isOpen? false, length=0, newEndIndex=0, (17829096, 17830279) isOpen? 
false, length=0, newEndIndex=0, (17830280, 17831209) isOpen? false, length=0, 
newEndIndex=0, (17831210, 17831887) isOpen? false, length=0, newEndIndex=0, 
(17831888, 17832491) isOpen? false, length=0, newEndIndex=0, (17832492, 
17833577) isOpen? false, length=0, newEndIndex=0, (17833578, 17834387) isOpen? 
false, length=0, newEndIndex=0, (17834388, 17835369) isOpen? false, length=0, 
newEndIndex=0, (17835370, 17836107) isOpen? false, length=0, newEndIndex=0, 
(17836108, 17837307) isOpen? false, length=0, newEndIndex=0, (17837308, 
17838140) isOpen? false, length=0, newEndIndex=0, (17838141, 17838591) isOpen? 
false, length=0, newEndIndex=0, (17838592, 17839411) isOpen? false, length=0, 
newEndIndex=0, (17839412, 17840613) isOpen? false, length=0, newEndIndex=0, 
(17840614, 17841672) isOpen? false, length=0, newEndIndex=0, (17841673, 
17842917) isOpen? false, length=0, newEndIndex=0, (17842918, 17843636) isOpen? 
false, length=0, newEndIndex=0, (17843637, 17844560) isOpen? false, length=0, 
newEndIndex=0, (17844561, 17847272) isOpen? false, length=0, newEndIndex=0, 
(17847273, 17847746) isOpen? false, length=0, newEndIndex=0, (17847747, 
17848169) isOpen? false, length=0, newEndIndex=0, (17848170, 17848631) isOpen? 
false, length=0, newEndIndex=0, (17848632, 17849078) isOpen? false, length=0, 
newEndIndex=0, (17849079, 17850026) isOpen? false, length=0, newEndIndex=0, 
(17850027, 17850490) isOpen? false, length=0, newEndIndex=0, (17850491, 
17851382) isOpen? false, length=0, newEndIndex=0, (17851383, 17852289) isOpen? 
false, length=0, newEndIndex=0, (17852290, 17853645) isOpen? false, length=0, 
newEndIndex=0, (17853646, 17854151) isOpen? false, length=0, newEndIndex=0, 
(17854152, 17855586) isOpen? false, length=0, newEndIndex=0, (17855587, 
17856772) isOpen? false, length=0, newEndIndex=0, (17856773, 17857571) isOpen? 
false, length=0, newEndIndex=0, (17857572, 17858043) isOpen? false, length=0, 
newEndIndex=0, (17858044, 17858598) isOpen? false, length=0, newEndIndex=0, 
(17858599, 17859922) isOpen? false, length=0, newEndIndex=0, (17859923, 
17860327) isOpen? false, length=0, newEndIndex=0, (17860328, 17861304) isOpen? 
false, length=0, newEndIndex=0, (17861305, 17862381) isOpen? false, length=0, 
newEndIndex=0, (17862382, 17863572) isOpen? false, length=0, newEndIndex=0, 
(17863573, 17864018) isOpen? false, length=0, newEndIndex=0, (17864019, 
17865114) isOpen? false, length=0, newEndIndex=0, (17865115, 17866739) isOpen? 
false, length=0, newEndIndex=0, (17866740, 17867551) isOpen? false, length=0, 
newEndIndex=0]
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
segmented.SegmentedRaftLogWorker - 1@group-47BEDE733167-SegmentedRaftLogWorker 
adds IO task PurgeLog:17867551
2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
segmented.SegmentedRaftLogWorker - added IO task (queueSize: 5) 
PurgeLog:17867551
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
raftlog.RaftLog - complete future of IO task WriteLog:17876749
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876749
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 4) 
WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876750
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 3) 
WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876751
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 2) 
WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876752
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 1) 
WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876753
2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 0) 
PurgeLog:17867551
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] INFO 
raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: purgeIndex: 
updateToMax old=17817215, new=17867551, updated? true
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
raftlog.RaftLog - complete future of IO task PurgeLog:17867551
2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
segmented.SegmentedRaftLogWorker - done IO task PurgeLog:17867551


> Follower state synchronization is blocked
> -----------------------------------------
>
>                 Key: RATIS-2116
>                 URL: https://issues.apache.org/jira/browse/RATIS-2116
>             Project: Ratis
>          Issue Type: Bug
>    Affects Versions: 3.0.0, 2.5.1, 3.0.1
>            Reporter: Haibo Sun
>            Priority: Major
>
> Using version 2.5.1, we have discovered that in some cases, the state 
> synchronization of the follower will be permanently blocked.
> Scenario: When the task queue of the SegmentedRaftLogWorker is the pattern 
> (WriteLog, WriteLog, ..., PurgeLog), the last WriteLog of 
> RaftServerImpl.appendEntries does not immediately flush data and complete the 
> result future, because there is a pending PurgeLog task in the queue. It 
> enqueues the result future to be completed after the latter WriteLog flushes 
> data. However, the "nioEventLoopGroup-3-1" thread is already blocked, and 
> will not add new WriteLog to the task queue of SegmentedRaftLogWorker. This 
> leads to a deadlock and causes the state synchronization to stop.
> I confirmed this by adding debug logs, detailed information is attached 
> below. This issue can be easily reproduced by increasing the frequency of 
> TakeSnapshot and PurgeLog operations. In addition, after checking the code in 
> the master branch, this issue still exists.
>  
> *jstack:*
> "nioEventLoopGroup-3-1" #58 prio=10 os_prio=0 tid=0x00007fc58400b800 
> nid=0x5493a waiting on condition [0x00007fc5b4f28000]
> java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park0(Native Method)
>  - parking to wait for <0x00007fd86a4685e8> (a 
> java.util.concurrent.CompletableFuture$Signaller)
> at sun.misc.Unsafe.park(Unsafe.java:1025)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:176)
> at 
> java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
> at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> at 
> java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
> at java.util.concurrent.CompletableFuture.join(CompletableFuture.java:1934)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntries(RaftServerImpl.java:1379)
> at 
> org.apache.ratis.server.impl.RaftServerProxy.appendEntries(RaftServerProxy.java:649)
> at 
> org.apache.ratis.netty.server.NettyRpcService.handle(NettyRpcService.java:231)
> at 
> org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:95)
> at 
> org.apache.ratis.netty.server.NettyRpcService$InboundHandler.channelRead0(NettyRpcService.java:91)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
> at 
> org.apache.ratis.thirdparty.io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
> at 
> org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:346)
> at 
> org.apache.ratis.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:318)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:444)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:412)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:440)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:420)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:788)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:724)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:650)
> at 
> org.apache.ratis.thirdparty.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:562)
> at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997)
> at 
> org.apache.ratis.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
> at 
> org.apache.ratis.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.lang.Thread.run(Thread.java:882)
>  
> *The debug logs which confirmed the cause:* 
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] INFO 
> raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: snapshotIndex: 
> updateIncreasingly 17818160 -> 17868160
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
> raftlog.RaftLog - purging segments:toTruncate: null
> toDelete: [(17817216, 17818372) isOpen? false, length=0, newEndIndex=0, 
> (17818373, 17819224) isOpen? false, length=0, newEndIndex=0, (17819225, 
> 17820474) isOpen? false, length=0, newEndIndex=0, (17820475, 17821368) 
> isOpen? false, length=0, newEndIndex=0, (17821369, 17822360) isOpen? false, 
> length=0, newEndIndex=0, (17822361, 17823347) isOpen? false, length=0, 
> newEndIndex=0, (17823348, 17824136) isOpen? false, length=0, newEndIndex=0, 
> (17824137, 17825440) isOpen? false, length=0, newEndIndex=0, (17825441, 
> 17826153) isOpen? false, length=0, newEndIndex=0, (17826154, 17826936) 
> isOpen? false, length=0, newEndIndex=0, (17826937, 17827393) isOpen? false, 
> length=0, newEndIndex=0, (17827394, 17828569) isOpen? false, length=0, 
> newEndIndex=0, (17828570, 17829095) isOpen? false, length=0, newEndIndex=0, 
> (17829096, 17830279) isOpen? false, length=0, newEndIndex=0, (17830280, 
> 17831209) isOpen? false, length=0, newEndIndex=0, (17831210, 17831887) 
> isOpen? false, length=0, newEndIndex=0, (17831888, 17832491) isOpen? false, 
> length=0, newEndIndex=0, (17832492, 17833577) isOpen? false, length=0, 
> newEndIndex=0, (17833578, 17834387) isOpen? false, length=0, newEndIndex=0, 
> (17834388, 17835369) isOpen? false, length=0, newEndIndex=0, (17835370, 
> 17836107) isOpen? false, length=0, newEndIndex=0, (17836108, 17837307) 
> isOpen? false, length=0, newEndIndex=0, (17837308, 17838140) isOpen? false, 
> length=0, newEndIndex=0, (17838141, 17838591) isOpen? false, length=0, 
> newEndIndex=0, (17838592, 17839411) isOpen? false, length=0, newEndIndex=0, 
> (17839412, 17840613) isOpen? false, length=0, newEndIndex=0, (17840614, 
> 17841672) isOpen? false, length=0, newEndIndex=0, (17841673, 17842917) 
> isOpen? false, length=0, newEndIndex=0, (17842918, 17843636) isOpen? false, 
> length=0, newEndIndex=0, (17843637, 17844560) isOpen? false, length=0, 
> newEndIndex=0, (17844561, 17847272) isOpen? false, length=0, newEndIndex=0, 
> (17847273, 17847746) isOpen? false, length=0, newEndIndex=0, (17847747, 
> 17848169) isOpen? false, length=0, newEndIndex=0, (17848170, 17848631) 
> isOpen? false, length=0, newEndIndex=0, (17848632, 17849078) isOpen? false, 
> length=0, newEndIndex=0, (17849079, 17850026) isOpen? false, length=0, 
> newEndIndex=0, (17850027, 17850490) isOpen? false, length=0, newEndIndex=0, 
> (17850491, 17851382) isOpen? false, length=0, newEndIndex=0, (17851383, 
> 17852289) isOpen? false, length=0, newEndIndex=0, (17852290, 17853645) 
> isOpen? false, length=0, newEndIndex=0, (17853646, 17854151) isOpen? false, 
> length=0, newEndIndex=0, (17854152, 17855586) isOpen? false, length=0, 
> newEndIndex=0, (17855587, 17856772) isOpen? false, length=0, newEndIndex=0, 
> (17856773, 17857571) isOpen? false, length=0, newEndIndex=0, (17857572, 
> 17858043) isOpen? false, length=0, newEndIndex=0, (17858044, 17858598) 
> isOpen? false, length=0, newEndIndex=0, (17858599, 17859922) isOpen? false, 
> length=0, newEndIndex=0, (17859923, 17860327) isOpen? false, length=0, 
> newEndIndex=0, (17860328, 17861304) isOpen? false, length=0, newEndIndex=0, 
> (17861305, 17862381) isOpen? false, length=0, newEndIndex=0, (17862382, 
> 17863572) isOpen? false, length=0, newEndIndex=0, (17863573, 17864018) 
> isOpen? false, length=0, newEndIndex=0, (17864019, 17865114) isOpen? false, 
> length=0, newEndIndex=0, (17865115, 17866739) isOpen? false, length=0, 
> newEndIndex=0, (17866740, 17867551) isOpen? false, length=0, newEndIndex=0]
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
> segmented.SegmentedRaftLogWorker - 
> 1@group-47BEDE733167-SegmentedRaftLogWorker adds IO task PurgeLog:17867551
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-StateMachineUpdater] DEBUG 
> segmented.SegmentedRaftLogWorker - added IO task (queueSize: 5) 
> PurgeLog:17867551
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> raftlog.RaftLog - complete future of IO task WriteLog:17876749
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876749
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 4) 
> WriteLog:17876750
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876750
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876750
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 3) 
> WriteLog:17876751
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876751
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876751
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 2) 
> WriteLog:17876752
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876752
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876752
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 1) 
> WriteLog:17876753
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - offer future of IO task WriteLog:17876753
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task WriteLog:17876753
> 2024-06-23 11:44:08,881 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - dequeued IO task (index: 0, queueSize: 0) 
> PurgeLog:17867551
> 2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] INFO 
> raftlog.RaftLog - 1@group-47BEDE733167-SegmentedRaftLog: purgeIndex: 
> updateToMax old=17817215, new=17867551, updated? true
> 2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> raftlog.RaftLog - complete future of IO task PurgeLog:17867551
> 2024-06-23 11:44:08,904 [1@group-47BEDE733167-SegmentedRaftLogWorker] DEBUG 
> segmented.SegmentedRaftLogWorker - done IO task PurgeLog:17867551



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

Reply via email to