You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2020/09/25 04:57:50 UTC

[incubator-ratis] branch master updated: RATIS-1074: GrpcLogAppender improperly decrease nextIndex to 1, which may trigger a installSnapshot request to follower (#207) Contributed by Glen Geng

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 50594a5  RATIS-1074: GrpcLogAppender improperly decrease nextIndex to 1, which may trigger a installSnapshot request to follower (#207)  Contributed by Glen Geng
50594a5 is described below

commit 50594a565cdd7d98dbf0f8c19aaf8e169154c7a4
Author: GlenGeng <gl...@tencent.com>
AuthorDate: Fri Sep 25 12:57:41 2020 +0800

    RATIS-1074: GrpcLogAppender improperly decrease nextIndex to 1, which may trigger a installSnapshot request to follower (#207)  Contributed by Glen Geng
---
 .../org/apache/ratis/grpc/server/GrpcLogAppender.java   | 17 ++++++++++++-----
 1 file changed, 12 insertions(+), 5 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 235a023..84d1a09 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
@@ -85,17 +85,24 @@ public class GrpcLogAppender extends LogAppender {
     return rpcService.getProxies().getProxy(getFollowerId());
   }
 
-  private synchronized void resetClient(AppendEntriesRequest request) {
+  private synchronized void resetClient(AppendEntriesRequest request, boolean onError) {
     rpcService.getProxies().resetProxy(getFollowerId());
     appendLogRequestObserver = null;
     firstResponseReceived = false;
 
     // clear the pending requests queue and reset the next index of follower
+    pendingRequests.clear();
+
     final long nextIndex = 1 + Optional.ofNullable(request)
         .map(AppendEntriesRequest::getPreviousLog)
         .map(TermIndex::getIndex)
         .orElseGet(getFollower()::getMatchIndex);
-    pendingRequests.clear();
+
+    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());
+      return;
+    }
     getFollower().decreaseNextIndex(nextIndex);
   }
 
@@ -316,13 +323,13 @@ public class GrpcLogAppender extends LogAppender {
       GrpcUtil.warn(LOG, () -> this + ": Failed appendEntries", t);
       grpcServerMetrics.onRequestRetry(); // Update try counter
       AppendEntriesRequest request = pendingRequests.remove(GrpcUtil.getCallId(t), GrpcUtil.isHeartbeat(t));
-      resetClient(request);
+      resetClient(request, true);
     }
 
     @Override
     public void onCompleted() {
       LOG.info("{}: follower responses appendEntries COMPLETED", this);
-      resetClient(null);
+      resetClient(null, false);
     }
 
     @Override
@@ -430,7 +437,7 @@ public class GrpcLogAppender extends LogAppender {
       }
       GrpcUtil.warn(LOG, () -> this + ": Failed InstallSnapshot", t);
       grpcServerMetrics.onRequestRetry(); // Update try counter
-      resetClient(null);
+      resetClient(null, true);
       close();
     }