You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by ms...@apache.org on 2018/01/22 07:29:59 UTC

incubator-ratis git commit: RATIS-153. AsyncAppendEntries misses updating followerstate at end of logsync. Contributed by Mukul Kumar Singh.

Repository: incubator-ratis
Updated Branches:
  refs/heads/master 9f93037a9 -> 0f7169db5


RATIS-153. AsyncAppendEntries misses updating followerstate at end of logsync. Contributed by Mukul Kumar Singh.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ratis/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ratis/commit/0f7169db
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ratis/tree/0f7169db
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ratis/diff/0f7169db

Branch: refs/heads/master
Commit: 0f7169db51ba15108c265b1e937e9daeaa085045
Parents: 9f93037
Author: Mukul Kumar Singh <ms...@apache.org>
Authored: Mon Jan 22 12:59:31 2018 +0530
Committer: Mukul Kumar Singh <ms...@apache.org>
Committed: Mon Jan 22 12:59:31 2018 +0530

----------------------------------------------------------------------
 .../ratis/server/impl/RaftServerImpl.java       | 22 ++++++++++++--------
 1 file changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ratis/blob/0f7169db/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
----------------------------------------------------------------------
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index b58ffdb..44499be 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -795,14 +795,6 @@ public class RaftServerImpl implements RaftServerProtocol, RaftServerAsynchronou
       CodeInjectionForTesting.execute(RaftLog.LOG_SYNC, getId(), null);
       nextIndex = entries[entries.length - 1].getIndex() + 1;
     }
-    synchronized (this) {
-      if (lifeCycle.getCurrentState() == RUNNING && isFollower()
-          && getState().getCurrentTerm() == currentTerm) {
-        // reset election timer to avoid punishing the leader for our own
-        // long disk writes
-        heartbeatMonitor.updateLastRpcTime(false);
-      }
-    }
     final AppendEntriesReplyProto reply = ServerProtoUtils.toAppendEntriesReplyProto(
         leaderId, getId(), groupId, currentTerm, nextIndex, SUCCESS);
     logAppendEntries(isHeartbeat,
@@ -810,7 +802,19 @@ public class RaftServerImpl implements RaftServerProtocol, RaftServerAsynchronou
             + ServerProtoUtils.toString(reply));
     return CompletableFuture
         .allOf(futures.toArray(new CompletableFuture[futures.size()]))
-        .thenApply(v -> reply);
+        .thenApply(v -> {
+          // reset election timer to avoid punishing the leader for our own
+          // long disk writes
+          synchronized (this) {
+            if (lifeCycle.getCurrentState() == RUNNING && isFollower()
+                && getState().getCurrentTerm() == currentTerm) {
+              // reset election timer to avoid punishing the leader for our own
+              // long disk writes
+              heartbeatMonitor.updateLastRpcTime(false);
+            }
+          }
+          return reply;
+        });
   }
 
   private boolean containPrevious(TermIndex previous) {