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

Xinhao GU edited comment on RATIS-2278 at 4/17/25 12:29 PM:
------------------------------------------------------------

Hi, [~szetszwo] 

*The test data above has been deleted. Below are the new test results, which 
can reproduce this issue. The specific process is as follows:*
 
*GrpcLogAppender-LogAppenderDaemon restarts after failure.*
h3. *Leader's behaviors*
 * {*}Leader sends request_0{*}{*}:{*}

 * 
 ** the AppendEntriesRequest to be sent is : cid=0,entriesCount=0
 * *Leader sends request_1:*

 * 
 ** the AppendEntriesRequest to be sent is : 
cid=1,entriesCount=146,entries=(t:1, i:0)...(t:1, i:145)
 ** nextIndex: updateIncreasingly 0 -> 146
 * *{{Leader receives response_0:}}*

 * 
 ** received the first reply 
5<-2#0:OK-t1,SUCCESS,nextIndex=143,followerCommit=137,matchIndex=-1 from 
follower, request=AppendEntriesRequest:cid=0,entriesCount=0
 * *Leader sends request_2:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=2,entriesCount=1,entry=(t:1, 
i:146)
 *** nextIndex: updateIncreasingly 146 -> 147
 * *Leader sends request_3:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=3,entriesCount=2,entries=(t:1, 
i:147)...(t:1, i:148)
 *** nextIndex: updateIncreasingly 147 -> 149
 * *{{Leader receives response_1:}}*

 * 
 ** received a reply 
5<-2#1:{*}FAIL{*}-t1,{*}INCONSISTENCY{*},nextIndex=138,followerCommit=137,matchIndex=-1
 from follower, 
request=AppendEntriesRequest:cid=1,entriesCount=146,entries=(t:1, i:0)...(t:1, 
i:145)
 ** as the first entry (index 0) already exists (snapshotIndex: -1, 
commitIndex: 137)
 ** setNextIndex nextIndex: updateUnconditionally 149 -> 138
 * *{{Leader receives response_2:}}*

 * 
 ** received a reply 
5<-2#2:{*}FAIL{*}-t1,INCONSISTENCY,nextIndex=143,followerCommit=137,matchIndex=-1
 from follower, request=null
 ** as previous log entry ((t:1, i:145)) is not found
 * *Leader sends request_4:*

 * 
 ** the AppendEntriesRequest to be sent is : 
cid=4,entriesCount=12,entries=(t:1, i:138)...(t:1, i:149)
 *** nextIndex: updateIncreasingly 138 -> 150
 * {color:#de350b}+_setNextIndex nextIndex: updateUnconditionally 150 -> 
143_+{color}

 * *Leader sends request_5:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=5,entriesCount=7,entries=(t:1, 
i:143)...(t:1, i:149)
 *** nextIndex: updateIncreasingly 143 -> 150
 * *{{Leader receives response_3:}}*

 * 
 ** received a reply 
5<-2#3:{*}FAIL{*}-t1,INCONSISTENCY,nextIndex=143,followerCommit=137,matchIndex=-1
 from follower, request=null
 ** as previous log entry ((t:1, i:146)) is not found
 ** setNextIndex nextIndex: updateUnconditionally 150 -> 143

h3. *Follower's behaviors*
 * *Follower received Leader's request_4*

 * 
 ** appendEntries* 5->2#4-t1,previous=(t:1, 
i:137),leaderCommit=148,initializing? true,entries: size=12, first=(t:1, 
i:138), METADATAENTRY(c:135)
 * *Request_4 passed checkInconsistentAppendEntries(), and these log entries 
are written into the RaftLog.*

 * 
 ** Start to append entry (t:1, i:143) into disk and cache
 ** ……

 * 
 ** Start to append entry (t:1, i:149) into disk and cache
 * *Follower received Leader's request_5*
 ** appendEntries* 5->2#5-t1,previous=(t:1, 
i:142),leaderCommit=148,initializing? true,entries: size=7, first=(t:1, i:143), 
STATEMACHINELOGENTRY
 * *The start index in request_5 conflicts with the last index in request_4*

 * 
 ** startIndex = 143 < nextIndex = 150.
 ** The reply is ERROR. 
 ** Notice leader to callback onError() and resetClient(), so the 
pendingRequest that was waiting is deleted.
 
{code:java}
2025-04-17 12:23:47,523 [grpc-default-executor-4] ERROR 
o.a.r.s.i.RaftServerImpl:1550 - 2@group-000100000004: Failed appendEntries* 
5->2#5-t1,previous=(t:1, i:142),leaderCommit=148,initializing? true,entries: 
size=7, first=(t:1, i:143), STATEMACHINELOGENTRY, 
StateMachineLogEntryProto:582@client-02CA3A4A0666 
java.lang.IllegalStateException: startIndex = 143 < nextIndex = 150
    at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
    at 
org.apache.ratis.server.impl.ServerImplUtils$NavigableIndices.append(ServerImplUtils.java:145)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendLog(RaftServerImpl.java:1693)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1657)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1548)
    at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:662)
    at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
    at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$29(RaftServerProxy.java:661)
    at 
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:995)
    at 
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2137)
    at 
org.apache.ratis.server.impl.RaftServerProxy.appendEntriesAsync(RaftServerProxy.java:661)
    at 
org.apache.ratis.grpc.server.GrpcServerProtocolService$1.process(GrpcServerProtocolService.java:323)
    at 
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:187)
    at 
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:334)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:319)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:834)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748) {code}


was (Author: JIRAUSER308993):
Hi, [~szetszwo] 

*The test data above has been deleted. Below are the new test results, which 
can reproduce this issue. The specific process is as follows:*
 
*GrpcLogAppender-LogAppenderDaemon restarts after failure.*
h3. *Leader's behaviors*
 * {*}Leader sends request_0{*}{*}:{*}

 * 
 ** the AppendEntriesRequest to be sent is : cid=0,entriesCount=0
 * *Leader sends request_1:*

 * 
 ** the AppendEntriesRequest to be sent is : 
cid=1,entriesCount=146,entries=(t:1, i:0)...(t:1, i:145)
 ** nextIndex: updateIncreasingly 0 -> 146
 * *{{Leader receives response_0:}}*

 * 
 ** received the first reply 
5<-2#0:OK-t1,SUCCESS,nextIndex=143,followerCommit=137,matchIndex=-1 from 
follower, request=AppendEntriesRequest:cid=0,entriesCount=0
 * *Leader sends request_2:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=2,entriesCount=1,entry=(t:1, 
i:146)
 *** nextIndex: updateIncreasingly 146 -> 147
 * *Leader sends request_3:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=3,entriesCount=2,entries=(t:1, 
i:147)...(t:1, i:148)
 *** nextIndex: updateIncreasingly 147 -> 149
 * *{{Leader receives response_1:}}*

 * 
 ** received a reply 
5<-2#1:{*}FAIL{*}-t1,{*}INCONSISTENCY{*},nextIndex=138,followerCommit=137,matchIndex=-1
 from follower, 
request=AppendEntriesRequest:cid=1,entriesCount=146,entries=(t:1, i:0)...(t:1, 
i:145)
 ** as the first entry (index 0) already exists (snapshotIndex: -1, 
commitIndex: 137)
 ** setNextIndex nextIndex: updateUnconditionally 149 -> 138
 * *{{Leader receives response_2:}}*

 * 
 ** received a reply 
5<-2#2:{*}FAIL{*}-t1,INCONSISTENCY,nextIndex=143,followerCommit=137,matchIndex=-1
 from follower, request=null
 ** as previous log entry ((t:1, i:145)) is not found
 * *Leader sends request_4:*

 * 
 ** the AppendEntriesRequest to be sent is : 
cid=4,entriesCount=12,entries=(t:1, i:138)...(t:1, i:149)
 *** nextIndex: updateIncreasingly 138 -> 150
 * {color:#de350b}+_setNextIndex nextIndex: updateUnconditionally 150 -> 
143_+{color}

 * *Leader sends request_5:*

 * 
 ** the AppendEntriesRequest to be sent is : cid=5,entriesCount=7,entries=(t:1, 
i:143)...(t:1, i:149)
 *** nextIndex: updateIncreasingly 143 -> 150
 * *{{Leader receives response_3:}}*

 * 
 ** received a reply 
5<-2#3:{*}FAIL{*}-t1,INCONSISTENCY,nextIndex=143,followerCommit=137,matchIndex=-1
 from follower, request=null
 ** as previous log entry ((t:1, i:146)) is not found
 ** setNextIndex nextIndex: updateUnconditionally 150 -> 143

h3. *Follower's behaviors*
 * *Follower received Leader's request_4*

 * 
 ** appendEntries* 5->2#4-t1,previous=(t:1, 
i:137),leaderCommit=148,initializing? true,entries: size=12, first=(t:1, 
i:138), METADATAENTRY(c:135)
 * *Request_4 passed checkInconsistentAppendEntries(), and these log entries 
are written into the RaftLog.*

 * 
 ** Start to append entry (t:1, i:143) into disk and cache
 ** ……

 * 
 ** Start to append entry (t:1, i:149) into disk and cache
 * *Follower received Leader's request_5*
 ** appendEntries* 5->2#5-t1,previous=(t:1, 
i:142),leaderCommit=148,initializing? true,entries: size=7, first=(t:1, i:143), 
STATEMACHINELOGENTRY
 * *The start index in request_5 conflicts with the last index in request_4*

 * 
 ** startIndex = 143 < nextIndex = 150
 
{code:java}
2025-04-17 12:23:47,523 [grpc-default-executor-4] ERROR 
o.a.r.s.i.RaftServerImpl:1550 - 2@group-000100000004: Failed appendEntries* 
5->2#5-t1,previous=(t:1, i:142),leaderCommit=148,initializing? true,entries: 
size=7, first=(t:1, i:143), STATEMACHINELOGENTRY, 
StateMachineLogEntryProto:582@client-02CA3A4A0666 
java.lang.IllegalStateException: startIndex = 143 < nextIndex = 150
    at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
    at 
org.apache.ratis.server.impl.ServerImplUtils$NavigableIndices.append(ServerImplUtils.java:145)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendLog(RaftServerImpl.java:1693)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1657)
    at 
org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1548)
    at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$28(RaftServerProxy.java:662)
    at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
    at 
org.apache.ratis.server.impl.RaftServerProxy.lambda$appendEntriesAsync$29(RaftServerProxy.java:661)
    at 
java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:995)
    at 
java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2137)
    at 
org.apache.ratis.server.impl.RaftServerProxy.appendEntriesAsync(RaftServerProxy.java:661)
    at 
org.apache.ratis.grpc.server.GrpcServerProtocolService$1.process(GrpcServerProtocolService.java:323)
    at 
org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:187)
    at 
org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:334)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:319)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:834)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
    at 
org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
    at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
    at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
    at java.lang.Thread.run(Thread.java:748) {code}

> Follower Fails to Append Entries Due to Index Validation Race Condition in 
> NavigableIndices
> -------------------------------------------------------------------------------------------
>
>                 Key: RATIS-2278
>                 URL: https://issues.apache.org/jira/browse/RATIS-2278
>             Project: Ratis
>          Issue Type: Bug
>          Components: Follower
>            Reporter: Xinhao GU
>            Assignee: Xinhao GU
>            Priority: Major
>             Fix For: 3.2.0
>
>         Attachments: image-2025-04-06-22-56-34-207.png, 
> image-2025-04-12-21-39-10-485.png, image.png, output.png
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> h2. *Problem Description*
> When a follower receives overlapping {{appendEntries}} requests from the 
> leader, it may throw an {{IllegalStateException}} during log index 
> validation. Specifically, the {{NavigableIndices}} class fails to handle 
> concurrent log appends properly, causing the follower to incorrectly reject 
> valid entries with an "index mismatch" error.
> h2. *Steps to Reproduce*
>  # {*}Cluster Setup{*}: Deploy a Raft group with at least 1 leader and 2 
> followers.
>  # {*}High Concurrency Scenario{*}: Trigger frequent {{appendEntries}} 
> requests from the leader to a follower.
>  # {*}Overlapping Requests{*}: Ensure two threads ({{{}thread-1{}}} and 
> {{{}thread-2{}}}) process {{appendEntries}} requests simultaneously:
>  ** {{thread-1}} appends logs with indices {{16093–16185}} and enters 
> {{state.getLog().append()}} but hasn’t invoked {{{}removeExisting(){}}}.
>  ** {{thread-2}} attempts to append logs starting at index {{16093}} (from a 
> new request) while {{thread-1}} is still processing.
>  # {*}Validation Failure{*}: The {{NavigableIndices.append()}} method detects 
> a gap between {{{}thread-1{}}}'s last index ({{{}16185{}}}) and 
> {{{}thread-2{}}}'s start index ({{{}16093{}}}), throwing an error.
> h2. *Expected Behavior*
> Concurrent {{appendEntries}} requests should be serialized or validated 
> atomically to prevent index gaps. Followers should accept logs as long as 
> they pass the initial {{ConsecutiveIndices.convert()}} check.
> h2. *Actual Behavior*
>  
> {code:java}
> 2025-03-04 05:44:14,946 [7-server-thread2] ERROR 
> o.a.r.s.i.RaftServerImpl:1488 - 7@group-000200000000: Failed appendEntries 
> 8->7#73-t49,previous=(t:3, i:16092),leaderCommit=16184,initializing? 
> false,entries: size=93, first=(t:4, i:16093), 
> CONFIGURATIONENTRY(current:id:"6"address:"172.16.2.14:10750"startupRole:FOLLOWER,
>  id:"7"address:"172.16.2.15:10750"startupRole:FOLLOWER, 
> id:"8"address:"172.16.2.11:10750"startupRole:FOLLOWER, old 
> java.lang.IllegalStateException: startIndex: expected == 16186 but computed 
> == 16093 at 
> org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
> at org.apache.ratis.util.Preconditions.assertSame(Preconditions.java:87)
> at 
> org.apache.ratis.server.impl.ServerImplUtils$NavigableIndices.append(ServerImplUtils.java:144)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendLog(RaftServerImpl.java:1631)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1596)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1486)
> at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$null$28(RaftServerProxy.java:646)
> at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:906)
> at 
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
> at java.base/java.lang.Thread.run(Thread.java:833)
> 2025-03-04 05:44:14,950 [grpc-default-executor-8] WARN 
> o.a.ratis.util.LogUtils:129 - 7: Failed APPEND_ENTRIES request 
> 8->7#73-t49,previous=(t:3, i:16092),leaderCommit=16184,initializing? 
> false,entries: size=93, first=(t:4, i:16093), 
> CONFIGURATIONENTRY(current:id:"6"address:"172.16.2.14:10750"startupRole:FOLLOWER,
>  id:"7"address:"172.16.2.15:10750"startupRole:FOLLOWER, 
> id:"8"address:"172.16.2.11:10750"startupRole:FOLLOWER, old 
> java.util.concurrent.CompletionException: java.io.IOException: 
> java.lang.IllegalStateException: startIndex: expected == 16186 but computed 
> == 16093
> at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:122)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.lambda$executeSubmitServerRequestAsync$10(RaftServerImpl.java:906)
> at 
> java.base/java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1768)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
> at java.base/java.lang.Thread.run(Thread.java:833)
> Caused by: java.io.IOException: java.lang.IllegalStateException: startIndex: 
> expected == 16186 but computed == 16093
> at org.apache.ratis.util.IOUtils.asIOException(IOUtils.java:56)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1490)
> at 
> org.apache.ratis.server.impl.RaftServerProxy.lambda$null$28(RaftServerProxy.java:646)
> at org.apache.ratis.util.JavaUtils.callAsUnchecked(JavaUtils.java:118)
> ... 5 common frames omitted
> Caused by: java.lang.IllegalStateException: startIndex: expected == 16186 but 
> computed == 16093
> at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:77)
> at org.apache.ratis.util.Preconditions.assertSame(Preconditions.java:87)
> at 
> org.apache.ratis.server.impl.ServerImplUtils$NavigableIndices.append(ServerImplUtils.java:144)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendLog(RaftServerImpl.java:1631)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1596)
> at 
> org.apache.ratis.server.impl.RaftServerImpl.appendEntriesAsync(RaftServerImpl.java:1486)
> ... 7 common frames omitted*{code}
> *!image-2025-04-06-22-56-34-207.png|width=1143,height=288!*
>  
> h2. *Root Cause Analysis*
>  # {*}Thread Race Condition{*}:
> The {{NavigableIndices}} class uses {{synchronized}} methods for {{append()}} 
> and {{{}removeExisting(){}}}, but *inter-method concurrency* is not handled. 
> If two threads interleave execution (e.g., {{thread-1}} appends logs but 
> hasn’t cleaned up {{map}} via {{{}removeExisting(){}}}, while {{thread-2}} 
> starts a new append), the index continuity check in {{append()}} fails 
> erroneously.
>  # {*}Redundant Validation{*}:
> The {{append()}} method revalidates log continuity using the {{{}map{}}}’s 
> last entry, even though {{ConsecutiveIndices.convert()}} already ensures 
> intra-request index continuity. This creates unnecessary 
> concurrency-sensitive checks.
> h2. *Proposed Solutions*
> h3. {*}Option 1{*}: Remove Redundant Validation in {{append()}}
> Modify {{NavigableIndices.append()}} to skip the startIndex check, relying 
> solely on {{convert()}} for validation.
> {*}Justification{*}:
>  * {{convert()}} already ensures all entries in a single request are 
> consecutive.
>  * {{removeExisting()}} guarantees cleanup after appending, making 
> cross-request checks redundant and race-prone.
> h3. {*}Option 2{*}: Global Lock for {{NavigableIndices'map}}
> Add a global lock for map to serialize all {{append()}} and 
> {{removeExisting()}} operations. However, this could introduce performance 
> bottlenecks.
>  



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

Reply via email to