This is an automated email from the ASF dual-hosted git repository.

szetszwo pushed a commit to branch branch-2_tmp
in repository https://gitbox.apache.org/repos/asf/ratis.git

commit c2e3bcca06ab3b566959133358a7f4399bd613bb
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Fri Feb 17 01:22:28 2023 -0800

    RATIS-1543. Log from GrpcLogAppender is confusing. (#821)
    
    (cherry picked from commit ca9fdd29e15b81464fd20d73a1d070a050bae30c)
---
 .../apache/ratis/grpc/server/GrpcLogAppender.java  | 23 +++++++++++++---------
 1 file changed, 14 insertions(+), 9 deletions(-)

diff --git 
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java 
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
index 125dd7dfa..7c742d91d 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
@@ -123,13 +123,14 @@ public class GrpcLogAppender extends LogAppenderBase {
       firstResponseReceived = false;
       // clear the pending requests queue and reset the next index of follower
       pendingRequests.clear();
+      final FollowerInfo f = getFollower();
       final long nextIndex = 1 + Optional.ofNullable(request)
           .map(AppendEntriesRequest::getPreviousLog)
           .map(TermIndex::getIndex)
-          .orElseGet(getFollower()::getMatchIndex);
-      if (onError && getFollower().getMatchIndex() == 0 && request == null) {
-        LOG.warn("{}: Leader has not got in touch with Follower {} yet, " +
-          "just keep nextIndex unchanged and retry.", this, getFollower());
+          .orElseGet(f::getMatchIndex);
+      if (onError && f.getMatchIndex() == 0 && request == null) {
+        LOG.warn("{}: Follower failed when matchIndex == 0, " +
+          " keep nextIndex ({}) unchanged and retry.", this, f.getNextIndex());
         return;
       }
       getFollower().decreaseNextIndex(nextIndex);
@@ -324,8 +325,10 @@ public class GrpcLogAppender extends LogAppenderBase {
     }
   }
 
-  private void increaseNextIndex(final long installedSnapshotIndex) {
-    getFollower().updateNextIndex(installedSnapshotIndex + 1);
+  private void increaseNextIndex(final long installedSnapshotIndex, Object 
reason) {
+    final long newNextIndex = installedSnapshotIndex + 1;
+    LOG.info("{}: updateNextIndex {} for {}", this, newNextIndex, reason);
+    getFollower().updateNextIndex(newNextIndex);
   }
 
   /**
@@ -381,12 +384,14 @@ public class GrpcLogAppender extends LogAppenderBase {
           break;
         case NOT_LEADER:
           grpcServerMetrics.onRequestNotLeader(getFollowerId().toString());
+          LOG.warn("{}: received {} reply with term {}", this, 
reply.getResult(), reply.getTerm());
           if (onFollowerTerm(reply.getTerm())) {
             return;
           }
           break;
         case INCONSISTENCY:
           grpcServerMetrics.onRequestInconsistency(getFollowerId().toString());
+          LOG.warn("{}: received {} reply with nextIndex {}", this, 
reply.getResult(), reply.getNextIndex());
           updateNextIndex(reply.getNextIndex());
           break;
         default:
@@ -518,7 +523,7 @@ public class GrpcLogAppender extends LogAppenderBase {
           getFollower().setSnapshotIndex(followerSnapshotIndex);
           getFollower().setAttemptedToInstallSnapshot();
           getLeaderState().onFollowerCommitIndex(getFollower(), 
followerSnapshotIndex);
-          increaseNextIndex(followerSnapshotIndex);
+          increaseNextIndex(followerSnapshotIndex, reply.getResult());
           removePending(reply);
           break;
         case NOT_LEADER:
@@ -535,7 +540,7 @@ public class GrpcLogAppender extends LogAppenderBase {
           getFollower().setSnapshotIndex(followerSnapshotIndex);
           getFollower().setAttemptedToInstallSnapshot();
           getLeaderState().onFollowerCommitIndex(getFollower(), 
followerSnapshotIndex);
-          increaseNextIndex(followerSnapshotIndex);
+          increaseNextIndex(followerSnapshotIndex, reply.getResult());
           onFollowerCatchup(followerSnapshotIndex);
           removePending(reply);
           break;
@@ -547,7 +552,7 @@ public class GrpcLogAppender extends LogAppenderBase {
           removePending(reply);
           break;
         case UNRECOGNIZED:
-          LOG.error("Unrecongnized the reply result {}: Leader is {}, follower 
is {}",
+          LOG.error("Unrecognized the reply result {}: Leader is {}, follower 
is {}",
               reply.getResult(), getServer().getId(), getFollowerId());
           break;
         default:

Reply via email to