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


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1645,7 +1647,27 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       return reply;
     });
   }
-  private CompletableFuture<Void> appendLog(List<LogEntryProto> entries) {
+
+    /**
+     * The listener checks whether it can become a follower.
+     *
+     * @return
+     */
+    private CompletableFuture<Void> checkAndUpdateListenerState() {

Review Comment:
   We should move the logic to changeToFollower(..).  Then it will work for 
other cases such as loading a snapshot.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1645,7 +1647,27 @@ leaderId, getMemberId(), currentTerm, followerCommit, 
inconsistencyReplyNextInde
       return reply;
     });
   }
-  private CompletableFuture<Void> appendLog(List<LogEntryProto> entries) {
+
+    /**
+     * The listener checks whether it can become a follower.
+     *
+     * @return
+     */
+    private CompletableFuture<Void> checkAndUpdateListenerState() {
+      CompletableFuture<Void> future = CompletableFuture.completedFuture(null);
+      if (role.getCurrentRole() == RaftPeerRole.LISTENER) {
+        if (state.getRaftConf().isStable() && 
state.getRaftConf().getPeer(getId()) != null) {

Review Comment:
   We should call state.getRaftConf() only once.  Otherwise the return value 
may have changed.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java:
##########
@@ -1570,6 +1570,7 @@ private CompletableFuture<AppendEntriesReplyProto> 
appendEntriesAsync(RaftPeerId
     final Optional<FollowerState> followerState;
     final Timekeeper.Context timer = 
raftServerMetrics.getFollowerAppendEntryTimer(isHeartbeat).time();
     final CompletableFuture<Void> future;
+    final CompletableFuture<Void> checkListenerFuture;

Review Comment:
   This is not needed anymore.



-- 
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