szetszwo commented on code in PR #807:
URL: https://github.com/apache/ratis/pull/807#discussion_r1092786533


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/TransferLeadership.java:
##########
@@ -81,10 +83,50 @@ public String toString() {
     this.server = server;
   }
 
+  private RaftPeerId getTransferee() {
+    return Optional.ofNullable(pending.get())
+        .map(r -> r.getRequest().getNewLeader()).orElse(null);
+  }
+
   boolean isSteppingDown() {
     return pending.get() != null;
   }
 
+  void onFollowerAppendEntriesReply(FollowerInfo follower) {
+    final RaftPeerId transferee = getTransferee();
+    // If TransferLeadership is in progress, and the transferee has just 
append some entries
+    if (follower.getPeer().getId().equals(transferee)) {
+      // If the transferee is up-to-date, send StartLeaderElection to it
+      server.getRole().getLeaderState().ifPresent(leaderState -> {

Review Comment:
   Pass `LeaderStateImple` as a parameter.  Then, we don't have to get it here.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/TransferLeadership.java:
##########
@@ -81,10 +83,50 @@ public String toString() {
     this.server = server;
   }
 
+  private RaftPeerId getTransferee() {
+    return Optional.ofNullable(pending.get())
+        .map(r -> r.getRequest().getNewLeader()).orElse(null);
+  }
+
   boolean isSteppingDown() {
     return pending.get() != null;
   }
 
+  void onFollowerAppendEntriesReply(FollowerInfo follower) {
+    final RaftPeerId transferee = getTransferee();
+    // If TransferLeadership is in progress, and the transferee has just 
append some entries
+    if (follower.getPeer().getId().equals(transferee)) {
+      // If the transferee is up-to-date, send StartLeaderElection to it
+      server.getRole().getLeaderState().ifPresent(leaderState -> {
+        if (leaderState.sendStartLeaderElection(follower)) {
+          LOG.info("{}: sent StartLeaderElection to transferee {} after 
received AppendEntriesResponse",
+              server.getMemberId(), transferee);
+        }
+      });
+    }
+  }
+
+  private void tryTransferLeadership(RaftPeerId transferee) {
+    server.getRole().getLeaderState().ifPresent(leaderState -> {

Review Comment:
   We should also pass `leaderState` as a parameter here.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -778,6 +778,7 @@ public void onFollowerSuccessAppendEntries(FollowerInfo 
follower) {
     } else {
       eventQueue.submit(checkStagingEvent);
     }
+    server.getTransferLeadership().onFollowerAppendEntriesReply(follower);

Review Comment:
   I wonder if we can "remember" the transferee follower so that we don't have 
to call it for all the other followers.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to