szetszwo commented on a change in pull request #376:
URL: https://github.com/apache/incubator-ratis/pull/376#discussion_r548871068



##########
File path: 
ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
##########
@@ -1387,6 +1388,41 @@ boolean resume() throws IOException {
     return true;
   }
 
+  @Override
+  public TimeoutNowReplyProto timeoutNow(TimeoutNowRequestProto request) 
throws IOException {
+    final RaftRpcRequestProto r = request.getServerRequest();
+    final RaftPeerId leaderId = RaftPeerId.valueOf(r.getRequestorId());
+    final RaftGroupId leaderGroupId = 
ProtoUtils.toRaftGroupId(r.getRaftGroupId());
+    final long leaderTerm = request.getLeaderTerm();
+    final TermIndex leaderLastEntry = 
TermIndex.valueOf(request.getLeaderLastEntry());
+
+    CodeInjectionForTesting.execute(TIMEOUT_NOW, getId(),
+        leaderId, leaderTerm, leaderLastEntry);
+
+    LOG.debug("{}: receive timeoutNow from:{}, leaderLastEntry:{},",
+        getMemberId(), leaderId, request.getLeaderLastEntry());
+
+    assertLifeCycleState(LifeCycle.States.RUNNING);
+    assertGroup(leaderId, leaderGroupId);
+
+    synchronized (this) {
+      if (!getInfo().isFollower()) {
+        LOG.warn("{} refused TimeoutNowRequest from {}, because role is:{}",
+            getMemberId(), leaderId, role.getCurrentRole());
+        return ServerProtoUtils.toTimeoutNowReplyProto(leaderId, 
getMemberId(), false);
+      }
+
+      if (ServerState.compareLog(state.getLastEntry(), leaderLastEntry) < 0) {
+        LOG.warn("{} refused TimeoutNowRequest from {}, because lastEntry:{} 
less than leaderEntry:{}",
+            getMemberId(), leaderId, leaderLastEntry, state.getLastEntry());
+        return ServerProtoUtils.toTimeoutNowReplyProto(leaderId, 
getMemberId(), false);
+      }
+
+      changeToCandidate();

Review comment:
       Similar to appendEntries, it should check if the request is come from 
the leader.

##########
File path: ratis-proto/src/main/proto/Raft.proto
##########
@@ -392,6 +392,16 @@ message TransferLeadershipRequestProto {
   RaftPeerProto newLeader = 2;
 }
 
+message TimeoutNowRequestProto {
+  RaftRpcRequestProto serverRequest = 1;
+  uint64 leaderTerm = 2;

Review comment:
       Could we remove this?  leaderLastEntry already has the term.

##########
File path: 
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java
##########
@@ -532,16 +534,28 @@ private void stepDown(long term, StepDownReason reason) {
     }
   }
 
-  private synchronized void yieldLeaderToHigherPriorityPeer(long term, 
TermIndex lastEntry) {
+  private synchronized void sendTimeoutNowToHigherPriorityPeer(RaftPeerId 
follower, TermIndex lastEntry) {
     ServerState state = server.getState();
     TermIndex currLastEntry = state.getLastEntry();
     if (ServerState.compareLog(currLastEntry, lastEntry) != 0) {
-      LOG.warn("{} can not stepDown because currLastEntry:{} did not match 
lastEntry:{}",
-          this, currLastEntry, lastEntry);
+      LOG.warn("{} can not send TimeoutNowRequest to follower:{} because 
currLastEntry:{} did not match lastEntry:{}",
+          this, follower, currLastEntry, lastEntry);
       return;
     }
 
-    stepDown(term, StepDownReason.HIGHER_PRIORITY);
+    final TimeoutNowRequestProto r = ServerProtoUtils.toTimeoutNowRequestProto(
+        server.getMemberId(), follower, state.getCurrentTerm(), lastEntry);
+    CompletableFuture.supplyAsync(() -> {
+      try {
+        TimeoutNowReplyProto replyProto = server.getServerRpc().timeoutNow(r);
+        if (replyProto.getServerReply().getSuccess()) {
+          LOG.warn("{} received failed reply of TimeoutNowRequest from 
follower:{}", this, follower);
+        }

Review comment:
       Let's print an info message with the reply is success.

##########
File path: 
ratis-server-api/src/main/java/org/apache/ratis/server/protocol/RaftServerProtocol.java
##########
@@ -34,4 +36,6 @@
   AppendEntriesReplyProto appendEntries(AppendEntriesRequestProto request) 
throws IOException;
 
   InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto 
request) throws IOException;
+
+  TimeoutNowReplyProto timeoutNow(TimeoutNowRequestProto request) throws 
IOException;

Review comment:
       Let's call this new method startLeaderElection.




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

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


Reply via email to