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

guangbao zhao updated RATIS-2145:
---------------------------------
    Description: 
We discovered a problem when writing tests with high concurrency. It often 
happens that a follower is running well and then triggers takeSnalshot.

The following is the relevant log.

follower: (as the follower log says, between 2024/08/22 20:18:14,044 and 
2024/08/22 20:21:57,058, no other logs appeared in the follower, but the 
follower election was not triggered, indicating that the leader gave The 
heartbeat sent by the follower is successful)
{code:java}
2024/08/22 20:18:13,987 [node1@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
22436696498 -> 22441096501
2024/08/22 20:18:13,999 [node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: 
node1@group-4F53D3317400-SegmentedRaftLogWorker: created new log segment 
/home/work/ssd1/lavafs/aktst-private/metaserver/metadata/ratis/23d5405d-0e30-3d56-9a77-4f53d3317400/current/log_inprogress_22441098615
2024/08/22 20:18:14,044 [node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
old=22432683959, new=22437078979, updated? true

2024/08/22 20:21:57,058 [grpc-default-executor-23] INFO 
com.xxx.RaftJournalManager: Received install snapshot notification from 
MetaStore leader: node3 with term index: (t:192, i:22441477801)
2024/08/22 20:21:57,059 [InstallSnapshotThread] INFO 
com.xxx.MetaStoreRatisSnapshotProvider: Downloading latest checkpoint from 
Leader MetaStore node3. Checkpoint address: leader:8170
2024/08/22 20:21:57,064 [grpc-default-executor-23] INFO 
org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
INSTALL_SNAPSHOT, lastRequest: node3->node1#0-t192,notify:(t:192, i:22441477801)
2024/08/22 20:21:57,065 [grpc-default-executor-23] INFO 
org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
INSTALL_SNAPSHOT, lastReply: null 
2024/08/22 20:21:57,067 [node1-server-thread55] INFO 
org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: Failed 
appendEntries as snapshot (22441477801) installation is in progress
2024/08/22 20:21:57,068 [node1-server-thread55] INFO 
org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: 
inconsistency entries. 
Reply:node3<-node1#19406445:FAIL-t192,INCONSISTENCY,nextIndex=22441098642,followerCommit=22441098595,matchIndex=-1{code}
leader:
{code:java}
2024/08/22 20:18:16,958 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867241,entriesCount=25,entries=(t:192, 
i:22441098598)...(t:192, i:22441098622)
2024/08/22 20:18:16,964 [timer3] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867246,entriesCount=1,entry=(t:192, 
i:22441098624)
2024/08/22 20:18:16,964 [timer6] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867247,entriesCount=1,entry=(t:192, 
i:22441098625)
2024/08/22 20:18:16,964 [timer7] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867245,entriesCount=1,entry=(t:192, 
i:22441098623)
2024/08/22 20:18:16,965 [timer3] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867255,entriesCount=1,entry=(t:192, 
i:22441098627)
2024/08/22 20:18:16,965 [timer7] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867253,entriesCount=1,entry=(t:192, 
i:22441098626)
2024/08/22 20:18:16,979 [timer0] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867406,entriesCount=13,entries=(t:192, 
i:22441098628)...(t:192, i:22441098640)
2024/08/22 20:18:16,981 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
i:22441098641)
2024/08/22 20:21:52,849 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
com.xxx.RaftJournalManager: Current LastApplied Index (t:192, i:22445496444)
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.impl.StateMachineUpdater: 
node3@group-4F53D3317400-StateMachineUpdater: Took a snapshot at index 
22445496444
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.impl.StateMachineUpdater: 
node3@group-4F53D3317400-StateMachineUpdater: snapshotIndex: updateIncreasingly 
22441096438 -> 22445496444
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node3@group-4F53D3317400-SegmentedRaftLog: purge 22441496502
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node3@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
22441096438 -> 22445496444
2024/08/22 20:21:57,021 
[node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
firstAvailable=(t:192, i:22441477801), followerNextIndex=22441098642 {code}
Because the leader did not receive the onNext callback within the 
requestTimeoutDuration(3s) time, the above warn log was 
generated.(raft.grpc.server.leader.outstanding.appends.max is set to 8)
{code:java}
scheduler.onTimeout(requestTimeoutDuration,
    () -> timeoutAppendRequest(request.getCallId(), request.isHeartbeat()),
    LOG, () -> "Timeout check failed for append entry request: " + request); 
{code}
and see through code:
{code:java}
private boolean haveTooManyPendingRequests() {
  final int size = pendingRequests.logRequestsSize();
  if (size == 0) {
    return false;
  } else if (size >= maxPendingRequestsNum) {
    return true;
  } else {
    // queue is non-empty and non-full
    return !replyState.isFirstReplyReceived();
  }
} {code}
When the size of pendingRequests.logRequestsSize() is >= 8, log entries will 
not be sent, only heartbeats will be sent.(This also verifies that there is a 
log gap in the follower log and no election is taking place)
{quote}leader: 

2024/08/22 20:18:16,981 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
i:22441098641)

2024/08/22 20:21:57,021 
[node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
firstAvailable=(t:192, i:22441477801), 
followerNextIndex={color:#FF0000}22441098642{color} 

follower:

2024/08/22 20:21:57,068 [node1-server-thread55] INFO 
org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: 
inconsistency entries. 
Reply:node3<-node1#19406445:FAIL-t192,INCONSISTENCY,nextIndex={color:#FF0000}22441098642{color},followerCommit=22441098595,matchIndex=-1
{quote}
And when the leader meets the purge log conditions, the leader will trigger 
notifyInstallSnapshot to the follower. At this time, the follower's nextIndex 
is 22441098642, and the last warn log is 22441098641 (exactly 22441098642 - 1), 
indicating that the follower has received the above warning log and updated the 
follower's nextIndex, but the logRequests of these 8 warns are still in 
logRequests , if it is the next 8 logRequests, then the corresponding warn log 
will appear in the following log, but I did not find it in the log.

As mentioned above, pendingRequests.logRequestsSize() >= maxPendingRequestsNum 
is always satisfied during the period of 2024/08/22 20:18:14,044 and 2024/08/22 
20:21:57,058, until the leader's firstAvailable > followerNextIndex
{quote} 
private void timeoutAppendRequest(long cid, boolean heartbeat) {
final AppendEntriesRequest pending = pendingRequests.handleTimeout(cid, 
heartbeat);
if (pending != null) {
final int errorCount = replyState.process(Event.TIMEOUT);
LOG.warn("{}: Timed out {}appendEntries, errorCount={}, request={}",
this, heartbeat ? "HEARTBEAT " : "", errorCount, pending);
grpcServerMetrics.onRequestTimeout(getFollowerId().toString(), heartbeat);
pending.stopRequestTimer();
}
}
 
{quote}
[~szetszwo] Is it possible to replace handleTimeout with the remove method 
here, so that it will not cause pendingRequests.logRequestsSize() >= 
maxPendingRequestsNum, causing the leader to keep sending heartbeats to the 
follower until notifyInstallSnapshot is triggered? leave the rest to the 
follower's checkInconsistentAppendEntries to ensure, do you think this is ok?

When this phenomenon occurs, the last log of the follower always stays at:
{quote}2024/08/22 20:18:14,044 
[node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
old=22432683959, new=22437078979, updated? true
{quote}
Or sometimes there are hidden bugs in the purge call logic that have not been 
discovered.

  was:
We discovered a problem when writing tests with high concurrency. It often 
happens that a follower is running well and then triggers takeSnalshot.

The following is the relevant log.

follower: (as the follower log says, between 2024/08/22 20:18:14,044 and 
2024/08/22 20:21:57,058, no other logs appeared in the follower, but the 
follower election was not triggered, indicating that the leader gave The 
heartbeat sent by the follower is successful)
{code:java}
2024/08/22 20:18:13,987 [node1@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
22436696498 -> 22441096501
2024/08/22 20:18:13,999 [node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: 
node1@group-4F53D3317400-SegmentedRaftLogWorker: created new log segment 
/home/work/ssd1/lavafs/aktst-private/metaserver/metadata/ratis/23d5405d-0e30-3d56-9a77-4f53d3317400/current/log_inprogress_22441098615
2024/08/22 20:18:14,044 [node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
old=22432683959, new=22437078979, updated? true

2024/08/22 20:21:57,058 [grpc-default-executor-23] INFO 
com.xxx.RaftJournalManager: Received install snapshot notification from 
MetaStore leader: node3 with term index: (t:192, i:22441477801)
2024/08/22 20:21:57,059 [InstallSnapshotThread] INFO 
com.xxx.MetaStoreRatisSnapshotProvider: Downloading latest checkpoint from 
Leader MetaStore node3. Checkpoint address: leader:8170
2024/08/22 20:21:57,064 [grpc-default-executor-23] INFO 
org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
INSTALL_SNAPSHOT, lastRequest: node3->node1#0-t192,notify:(t:192, i:22441477801)
2024/08/22 20:21:57,065 [grpc-default-executor-23] INFO 
org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
INSTALL_SNAPSHOT, lastReply: null {code}
leader:
{code:java}
2024/08/22 20:18:16,958 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867241,entriesCount=25,entries=(t:192, 
i:22441098598)...(t:192, i:22441098622)
2024/08/22 20:18:16,964 [timer3] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867246,entriesCount=1,entry=(t:192, 
i:22441098624)
2024/08/22 20:18:16,964 [timer6] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867247,entriesCount=1,entry=(t:192, 
i:22441098625)
2024/08/22 20:18:16,964 [timer7] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867245,entriesCount=1,entry=(t:192, 
i:22441098623)
2024/08/22 20:18:16,965 [timer3] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867255,entriesCount=1,entry=(t:192, 
i:22441098627)
2024/08/22 20:18:16,965 [timer7] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867253,entriesCount=1,entry=(t:192, 
i:22441098626)
2024/08/22 20:18:16,979 [timer0] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867406,entriesCount=13,entries=(t:192, 
i:22441098628)...(t:192, i:22441098640)
2024/08/22 20:18:16,981 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
i:22441098641)
2024/08/22 20:21:52,849 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
com.xxx.RaftJournalManager: Current LastApplied Index (t:192, i:22445496444)
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.impl.StateMachineUpdater: 
node3@group-4F53D3317400-StateMachineUpdater: Took a snapshot at index 
22445496444
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.impl.StateMachineUpdater: 
node3@group-4F53D3317400-StateMachineUpdater: snapshotIndex: updateIncreasingly 
22441096438 -> 22445496444
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node3@group-4F53D3317400-SegmentedRaftLog: purge 22441496502
2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node3@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
22441096438 -> 22445496444
2024/08/22 20:21:57,021 
[node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
firstAvailable=(t:192, i:22441477801), followerNextIndex=22441098642 {code}
Because the leader did not receive the onNext callback within the 
requestTimeoutDuration(3s) time, the above warn log was 
generated.(raft.grpc.server.leader.outstanding.appends.max is set to 8)
{code:java}
scheduler.onTimeout(requestTimeoutDuration,
    () -> timeoutAppendRequest(request.getCallId(), request.isHeartbeat()),
    LOG, () -> "Timeout check failed for append entry request: " + request); 
{code}
and see through code:
{code:java}
private boolean haveTooManyPendingRequests() {
  final int size = pendingRequests.logRequestsSize();
  if (size == 0) {
    return false;
  } else if (size >= maxPendingRequestsNum) {
    return true;
  } else {
    // queue is non-empty and non-full
    return !replyState.isFirstReplyReceived();
  }
} {code}
When the size of pendingRequests.logRequestsSize() is >= 8, log entries will 
not be sent, only heartbeats will be sent.(This also verifies that there is a 
log gap in the follower log and no election is taking place)
{quote} 2024/08/22 20:18:16,981 [timer5] WARN 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
errorCount=1, 
request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
i:22441098641)

2024/08/22 20:21:57,021 
[node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
org.apache.ratis.grpc.server.GrpcLogAppender: 
node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
firstAvailable=(t:192, i:22441477801), followerNextIndex=22441098642 
{quote}
And when the leader meets the purge log conditions, the leader will trigger 
notifyInstallSnapshot to the follower. At this time, the follower's nextIndex 
is 22441098642, and the last warn log is 22441098641 (exactly 22441098642 - 1), 
indicating that the follower has received the above warning log and updated the 
follower's nextIndex, but the logRequests of these 8 warns are still in 
logRequests , if it is the next 8 logRequests, then the corresponding warn log 
will appear in the following log, but I did not find it in the log.

As mentioned above, pendingRequests.logRequestsSize() >= maxPendingRequestsNum 
is always satisfied during the period of 2024/08/22 20:18:14,044 and 2024/08/22 
20:21:57,058, until the leader's firstAvailable > followerNextIndex
{quote} 
private void timeoutAppendRequest(long cid, boolean heartbeat) {
final AppendEntriesRequest pending = pendingRequests.handleTimeout(cid, 
heartbeat);
if (pending != null) {
final int errorCount = replyState.process(Event.TIMEOUT);
LOG.warn("{}: Timed out {}appendEntries, errorCount={}, request={}",
this, heartbeat ? "HEARTBEAT " : "", errorCount, pending);
grpcServerMetrics.onRequestTimeout(getFollowerId().toString(), heartbeat);
pending.stopRequestTimer();
}
}
 
{quote}
[~szetszwo] Is it possible to replace handleTimeout with the remove method 
here, so that it will not cause pendingRequests.logRequestsSize() >= 
maxPendingRequestsNum, causing the leader to keep sending heartbeats to the 
follower until notifyInstallSnapshot is triggered? leave the rest to the 
follower's checkInconsistentAppendEntries to ensure, do you think this is ok?

When this phenomenon occurs, the last log of the follower always stays at:
{quote}2024/08/22 20:18:14,044 
[node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
org.apache.ratis.server.raftlog.RaftLog: 
node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
old=22432683959, new=22437078979, updated? true
{quote}
Or sometimes there are hidden bugs in the purge call logic that have not been 
discovered.


>  Follower hangs until the next trigger to take a snapshot
> ---------------------------------------------------------
>
>                 Key: RATIS-2145
>                 URL: https://issues.apache.org/jira/browse/RATIS-2145
>             Project: Ratis
>          Issue Type: Bug
>          Components: gRPC
>    Affects Versions: 3.0.1
>            Reporter: guangbao zhao
>            Priority: Major
>
> We discovered a problem when writing tests with high concurrency. It often 
> happens that a follower is running well and then triggers takeSnalshot.
> The following is the relevant log.
> follower: (as the follower log says, between 2024/08/22 20:18:14,044 and 
> 2024/08/22 20:21:57,058, no other logs appeared in the follower, but the 
> follower election was not triggered, indicating that the leader gave The 
> heartbeat sent by the follower is successful)
> {code:java}
> 2024/08/22 20:18:13,987 [node1@group-4F53D3317400-StateMachineUpdater] INFO 
> org.apache.ratis.server.raftlog.RaftLog: 
> node1@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
> 22436696498 -> 22441096501
> 2024/08/22 20:18:13,999 [node1@group-4F53D3317400-SegmentedRaftLogWorker] 
> INFO org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogWorker: 
> node1@group-4F53D3317400-SegmentedRaftLogWorker: created new log segment 
> /home/work/ssd1/lavafs/aktst-private/metaserver/metadata/ratis/23d5405d-0e30-3d56-9a77-4f53d3317400/current/log_inprogress_22441098615
> 2024/08/22 20:18:14,044 [node1@group-4F53D3317400-SegmentedRaftLogWorker] 
> INFO org.apache.ratis.server.raftlog.RaftLog: 
> node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
> old=22432683959, new=22437078979, updated? true
> 2024/08/22 20:21:57,058 [grpc-default-executor-23] INFO 
> com.xxx.RaftJournalManager: Received install snapshot notification from 
> MetaStore leader: node3 with term index: (t:192, i:22441477801)
> 2024/08/22 20:21:57,059 [InstallSnapshotThread] INFO 
> com.xxx.MetaStoreRatisSnapshotProvider: Downloading latest checkpoint from 
> Leader MetaStore node3. Checkpoint address: leader:8170
> 2024/08/22 20:21:57,064 [grpc-default-executor-23] INFO 
> org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
> INSTALL_SNAPSHOT, lastRequest: node3->node1#0-t192,notify:(t:192, 
> i:22441477801)
> 2024/08/22 20:21:57,065 [grpc-default-executor-23] INFO 
> org.apache.ratis.grpc.server.GrpcServerProtocolService: node1: Completed 
> INSTALL_SNAPSHOT, lastReply: null 
> 2024/08/22 20:21:57,067 [node1-server-thread55] INFO 
> org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: Failed 
> appendEntries as snapshot (22441477801) installation is in progress
> 2024/08/22 20:21:57,068 [node1-server-thread55] INFO 
> org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: 
> inconsistency entries. 
> Reply:node3<-node1#19406445:FAIL-t192,INCONSISTENCY,nextIndex=22441098642,followerCommit=22441098595,matchIndex=-1{code}
> leader:
> {code:java}
> 2024/08/22 20:18:16,958 [timer5] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867241,entriesCount=25,entries=(t:192, 
> i:22441098598)...(t:192, i:22441098622)
> 2024/08/22 20:18:16,964 [timer3] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867246,entriesCount=1,entry=(t:192, 
> i:22441098624)
> 2024/08/22 20:18:16,964 [timer6] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867247,entriesCount=1,entry=(t:192, 
> i:22441098625)
> 2024/08/22 20:18:16,964 [timer7] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867245,entriesCount=1,entry=(t:192, 
> i:22441098623)
> 2024/08/22 20:18:16,965 [timer3] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867255,entriesCount=1,entry=(t:192, 
> i:22441098627)
> 2024/08/22 20:18:16,965 [timer7] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867253,entriesCount=1,entry=(t:192, 
> i:22441098626)
> 2024/08/22 20:18:16,979 [timer0] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867406,entriesCount=13,entries=(t:192, 
> i:22441098628)...(t:192, i:22441098640)
> 2024/08/22 20:18:16,981 [timer5] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
> i:22441098641)
> 2024/08/22 20:21:52,849 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
> com.xxx.RaftJournalManager: Current LastApplied Index (t:192, i:22445496444)
> 2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
> org.apache.ratis.server.impl.StateMachineUpdater: 
> node3@group-4F53D3317400-StateMachineUpdater: Took a snapshot at index 
> 22445496444
> 2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
> org.apache.ratis.server.impl.StateMachineUpdater: 
> node3@group-4F53D3317400-StateMachineUpdater: snapshotIndex: 
> updateIncreasingly 22441096438 -> 22445496444
> 2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
> org.apache.ratis.server.raftlog.RaftLog: 
> node3@group-4F53D3317400-SegmentedRaftLog: purge 22441496502
> 2024/08/22 20:21:57,021 [node3@group-4F53D3317400-StateMachineUpdater] INFO 
> org.apache.ratis.server.raftlog.RaftLog: 
> node3@group-4F53D3317400-SegmentedRaftLog: snapshotIndex: updateIncreasingly 
> 22441096438 -> 22445496444
> 2024/08/22 20:21:57,021 
> [node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
> firstAvailable=(t:192, i:22441477801), followerNextIndex=22441098642 {code}
> Because the leader did not receive the onNext callback within the 
> requestTimeoutDuration(3s) time, the above warn log was 
> generated.(raft.grpc.server.leader.outstanding.appends.max is set to 8)
> {code:java}
> scheduler.onTimeout(requestTimeoutDuration,
>     () -> timeoutAppendRequest(request.getCallId(), request.isHeartbeat()),
>     LOG, () -> "Timeout check failed for append entry request: " + request); 
> {code}
> and see through code:
> {code:java}
> private boolean haveTooManyPendingRequests() {
>   final int size = pendingRequests.logRequestsSize();
>   if (size == 0) {
>     return false;
>   } else if (size >= maxPendingRequestsNum) {
>     return true;
>   } else {
>     // queue is non-empty and non-full
>     return !replyState.isFirstReplyReceived();
>   }
> } {code}
> When the size of pendingRequests.logRequestsSize() is >= 8, log entries will 
> not be sent, only heartbeats will be sent.(This also verifies that there is a 
> log gap in the follower log and no election is taking place)
> {quote}leader: 
> 2024/08/22 20:18:16,981 [timer5] WARN 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: Timed out appendEntries, 
> errorCount=1, 
> request=AppendEntriesRequest:cid=16867411,entriesCount=1,entry=(t:192, 
> i:22441098641)
> 2024/08/22 20:21:57,021 
> [node3@group-4F53D3317400->node1-GrpcLogAppender-LogAppenderDaemon] INFO 
> org.apache.ratis.grpc.server.GrpcLogAppender: 
> node3@group-4F53D3317400->node1-GrpcLogAppender: notifyInstallSnapshot with 
> firstAvailable=(t:192, i:22441477801), 
> followerNextIndex={color:#FF0000}22441098642{color} 
> follower:
> 2024/08/22 20:21:57,068 [node1-server-thread55] INFO 
> org.apache.ratis.server.RaftServer$Division: node1@group-4F53D3317400: 
> inconsistency entries. 
> Reply:node3<-node1#19406445:FAIL-t192,INCONSISTENCY,nextIndex={color:#FF0000}22441098642{color},followerCommit=22441098595,matchIndex=-1
> {quote}
> And when the leader meets the purge log conditions, the leader will trigger 
> notifyInstallSnapshot to the follower. At this time, the follower's nextIndex 
> is 22441098642, and the last warn log is 22441098641 (exactly 22441098642 - 
> 1), indicating that the follower has received the above warning log and 
> updated the follower's nextIndex, but the logRequests of these 8 warns are 
> still in logRequests , if it is the next 8 logRequests, then the 
> corresponding warn log will appear in the following log, but I did not find 
> it in the log.
> As mentioned above, pendingRequests.logRequestsSize() >= 
> maxPendingRequestsNum is always satisfied during the period of 2024/08/22 
> 20:18:14,044 and 2024/08/22 20:21:57,058, until the leader's firstAvailable > 
> followerNextIndex
> {quote} 
> private void timeoutAppendRequest(long cid, boolean heartbeat) {
> final AppendEntriesRequest pending = pendingRequests.handleTimeout(cid, 
> heartbeat);
> if (pending != null) {
> final int errorCount = replyState.process(Event.TIMEOUT);
> LOG.warn("{}: Timed out {}appendEntries, errorCount={}, request={}",
> this, heartbeat ? "HEARTBEAT " : "", errorCount, pending);
> grpcServerMetrics.onRequestTimeout(getFollowerId().toString(), heartbeat);
> pending.stopRequestTimer();
> }
> }
>  
> {quote}
> [~szetszwo] Is it possible to replace handleTimeout with the remove method 
> here, so that it will not cause pendingRequests.logRequestsSize() >= 
> maxPendingRequestsNum, causing the leader to keep sending heartbeats to the 
> follower until notifyInstallSnapshot is triggered? leave the rest to the 
> follower's checkInconsistentAppendEntries to ensure, do you think this is ok?
> When this phenomenon occurs, the last log of the follower always stays at:
> {quote}2024/08/22 20:18:14,044 
> [node1@group-4F53D3317400-SegmentedRaftLogWorker] INFO 
> org.apache.ratis.server.raftlog.RaftLog: 
> node1@group-4F53D3317400-SegmentedRaftLog: purgeIndex: updateToMax 
> old=22432683959, new=22437078979, updated? true
> {quote}
> Or sometimes there are hidden bugs in the purge call logic that have not been 
> discovered.



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

Reply via email to