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

Sammi Chen commented on RATIS-946:
----------------------------------

Hi [~szetszwo], with the current inconsistent index handling in RATIS, is this 
problem resolved? 

> GrpcLogAppender fails to fix failed AppendEntries with inconsistency in case 
> of message losses
> ----------------------------------------------------------------------------------------------
>
>                 Key: RATIS-946
>                 URL: https://issues.apache.org/jira/browse/RATIS-946
>             Project: Ratis
>          Issue Type: Bug
>          Components: gRPC
>    Affects Versions: 1.0.0
>            Reporter: Burcu Ozkan
>            Priority: Major
>         Attachments: logs.txt, messages.txt
>
>
> I am testing fault tolerance of Ratis, more specifically whether it can 
> tolerate random message losses. Simply, I drop some of the messages and do 
> not deliver them to the recipient. 
> In some tests where the servers have inconsistent logs, I observe executions 
> in which the follower fails to synchronize with the leader.  The servers 
> indefinitely exchange AppendEntries request and its failing reply, logging 
> the following messages repeatedly:
> {noformat}
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 19 -> 17
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1097)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries as previous log entry ((t:20, 
> i:16)) is not found
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308912:FAIL,INCONSISTENCY,nextIndex:16,term:20,followerCommit:16
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 19 -> 16 
> {noformat}
> This occurs when gRPC adapter is used, I did not observe the problem using 
> Netty.
> Here is the part of the logs before the repeated messages:
> {noformat}
> ...
> impl.RoleInfo (RoleInfo.java:updateAndGet(143)) - s0: start FollowerState
> impl.RaftServerImpl (ServerState.java:setLeader(255)) - 
> s0@group-AD42E7C24DDE: change Leader from null to s1 at term 20 for 
> appendEntries, leader elected after 3ms
> client.GrpcClientProtocolService 
> (GrpcClientProtocolService.java:lambda$processClientRequest$0(283)) - Failed 
> RaftClientRequest:client-A0592557AEC3->s0@group-AD42E7C24DDE, cid=7, seq=0, 
> RW, Message:6d3130, 
> reply=RaftClientReply:client-A0592557AEC3->s0@group-AD42E7C24DDE, cid=7, 
> FAILED org.apache.ratis.protocol.NotLeaderException: Server 
> s0@group-AD42E7C24DDE is not the leader, logIndex=0, commits[s0:c14, s1:c13, 
> s2:c13]
> client.GrpcClientProtocolService 
> (GrpcClientProtocolService.java:lambda$processClientRequest$0(283)) - Failed 
> RaftClientRequest:client-3A32B89816B9->s0@group-AD42E7C24DDE, cid=8, seq=0, 
> RW, Message:6d3131, 
> reply=RaftClientReply:client-3A32B89816B9->s0@group-AD42E7C24DDE, cid=8, 
> FAILED org.apache.ratis.protocol.NotLeaderException: Server 
> s0@group-AD42E7C24DDE is not the leader, logIndex=0, commits[s0:c14, s1:c13, 
> s2:c13]
> client.GrpcClientProtocolService 
> (GrpcClientProtocolService.java:lambda$processClientRequest$0(283)) - Failed 
> RaftClientRequest:client-73072E12695E->s0@group-AD42E7C24DDE, cid=6, seq=0, 
> RW, Message:6d3132, 
> reply=RaftClientReply:client-73072E12695E->s0@group-AD42E7C24DDE, cid=6, 
> FAILED org.apache.ratis.protocol.NotLeaderException: Server 
> s0@group-AD42E7C24DDE is not the leader, logIndex=0, commits[s0:c14, s1:c13, 
> s2:c13]
> server.GrpcLogAppender (GrpcLogAppender.java:onCompleted(321)) - 
> s0@group-AD42E7C24DDE->s1-AppendLogResponseHandler: follower responses 
> appendEntries COMPLETED
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1088)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries: the first entry (index 15) is 
> already committed (commit index: 16)
> statemachine.SimpleStateMachine4Testing 
> (SimpleStateMachine4Testing.java:put(200)) - s0: put 15, m11 -> (t:19, i:15), 
> STATEMACHINELOGENTRY, client-3A32B89816B9, cid=8
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308907:FAIL,INCONSISTENCY,nextIndex:17,term:20,followerCommit:16
> statemachine.SimpleStateMachine4Testing 
> (SimpleStateMachine4Testing.java:put(200)) - s0: put 16, m10 -> (t:19, i:16), 
> STATEMACHINELOGENTRY, client-A0592557AEC3, cid=7
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 17 -> 17
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s0@group-AD42E7C24DDE->s2: nextIndex: updateUnconditionally 18 -> 15
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s0@group-AD42E7C24DDE->s1: nextIndex: updateUnconditionally 18 -> 15
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1097)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries as previous log entry ((t:20, 
> i:16)) is not found
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308908:FAIL,INCONSISTENCY,nextIndex:16,term:20,followerCommit:16
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 17 -> 16
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1088)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries: the first entry (index 16) is 
> already committed (commit index: 16)
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308909:FAIL,INCONSISTENCY,nextIndex:17,term:20,followerCommit:16
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 17 -> 17
> [ForkJoinPool.commonPool-worker-3] INFO client.GrpcClientProtocolService 
> (GrpcClientProtocolService.java:lambda$processClientRequest$0(283)) - Failed 
> RaftClientRequest:client-73072E12695E->s2@group-AD42E7C24DDE, cid=6, seq=0, 
> RW, Message:6d3132, 
> reply=RaftClientReply:client-73072E12695E->s2@group-AD42E7C24DDE, cid=6, 
> FAILED org.apache.ratis.protocol.NotLeaderException: Server 
> s2@group-AD42E7C24DDE is not the leader s1:0.0.0.0:49936, logIndex=0, 
> commits[s2:c16, s0:c16, s1:c16]
> [ForkJoinPool.commonPool-worker-1] INFO client.GrpcClientProtocolService 
> (GrpcClientProtocolService.java:lambda$processClientRequest$0(283)) - Failed 
> RaftClientRequest:client-3A32B89816B9->s2@group-AD42E7C24DDE, cid=8, seq=0, 
> RW, Message:6d3131, 
> reply=RaftClientReply:client-3A32B89816B9->s2@group-AD42E7C24DDE, cid=8, 
> FAILED org.apache.ratis.protocol.NotLeaderException: Server 
> s2@group-AD42E7C24DDE is not the leader s1:0.0.0.0:49936, logIndex=0, 
> commits[s2:c16, s0:c16, s1:c16]
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1097)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries as previous log entry ((t:20, 
> i:16)) is not found
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308910:FAIL,INCONSISTENCY,nextIndex:16,term:20,followerCommit:16
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 18 -> 16
> statemachine.SimpleStateMachine4Testing 
> (SimpleStateMachine4Testing.java:put(200)) - s1: put 17, m10 -> (t:20, i:17), 
> STATEMACHINELOGENTRY, client-A0592557AEC3, cid=7
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1088)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries: the first entry (index 16) is 
> already committed (commit index: 16)
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308911:FAIL,INCONSISTENCY,nextIndex:17,term:20,followerCommit:16
> statemachine.SimpleStateMachine4Testing 
> (SimpleStateMachine4Testing.java:put(200)) - s2: put 17, m10 -> (t:20, i:17), 
> STATEMACHINELOGENTRY, client-A0592557AEC3, cid=7
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 19 -> 17
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1097)) - 
> s0@group-AD42E7C24DDE: Failed appendEntries as previous log entry ((t:20, 
> i:16)) is not found
> impl.RaftServerImpl 
> (RaftServerImpl.java:checkInconsistentAppendEntries(1063)) - 
> s0@group-AD42E7C24DDE: inconsistency entries. 
> Reply:s1<-s0#308912:FAIL,INCONSISTENCY,nextIndex:16,term:20,followerCommit:16
> impl.FollowerInfo (FollowerInfo.java:lambda$new$0(51)) - 
> s1@group-AD42E7C24DDE->s0: nextIndex: updateUnconditionally 19 -> 16
> ..
> {noformat}
> I attached a file listing the messages sent or dropped in the execution. Each 
> line lists the sender/receiver of the message together with some message 
> information, and the number of times the message is sent/received. The 
> messages marked with "-D" are dropped. You can also find the full log of the 
> execution in the attachment.



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

Reply via email to