You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sh...@apache.org on 2021/03/17 12:43:02 UTC

[ratis] branch master updated: RATIS-1326. NotifyInstallSnapshot during SetConfiguration has leader info missing. (#449)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new dd0fad4  RATIS-1326. NotifyInstallSnapshot during SetConfiguration has leader info missing. (#449)
dd0fad4 is described below

commit dd0fad479ca9b9a67cd0856999729179fa1fd49d
Author: bshashikant <sh...@apache.org>
AuthorDate: Wed Mar 17 18:12:54 2021 +0530

    RATIS-1326. NotifyInstallSnapshot during SetConfiguration has leader info missing. (#449)
---
 .../org/apache/ratis/server/impl/RaftServerImpl.java     | 16 +++++++++-------
 .../apache/ratis/InstallSnapshotNotificationTests.java   |  5 ++++-
 2 files changed, 13 insertions(+), 8 deletions(-)

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 f932fbb..01046f9 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
@@ -526,13 +526,15 @@ class RaftServerImpl implements RaftServer.Division,
       break;
 
     case FOLLOWER:
-      role.getFollowerState().ifPresent(fs -> {
-        final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto(
-            getRaftConf().getPeer(state.getLeaderId()), fs.getLastRpcTime().elapsedTimeMs());
-        roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder()
-            .setLeaderInfo(leaderInfo)
-            .setOutstandingOp(fs.getOutstandingOp()));
-      });
+      final Optional<FollowerState> fs = role.getFollowerState();
+      final ServerRpcProto leaderInfo = ServerProtoUtils.toServerRpcProto(
+        getRaftConf().getPeer(state.getLeaderId()),
+        fs.map(FollowerState::getLastRpcTime).map(Timestamp::elapsedTimeMs).orElse(0L));
+      // FollowerState can be null while adding a new peer as it is not
+      // a voting member yet
+      roleInfo.setFollowerInfo(FollowerInfoProto.newBuilder()
+        .setLeaderInfo(leaderInfo)
+        .setOutstandingOp(fs.map(FollowerState::getOutstandingOp).orElse(0)));
       break;
 
     case LEADER:
diff --git a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
index a4c25da..c690ea1 100644
--- a/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/InstallSnapshotNotificationTests.java
@@ -86,7 +86,10 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC
     public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader(
         RaftProtos.RoleInfoProto roleInfoProto,
         TermIndex termIndex) {
-
+      if (!roleInfoProto.getFollowerInfo().hasLeaderInfo()) {
+        return JavaUtils.completeExceptionally(new IOException("Failed " +
+          "notifyInstallSnapshotFromLeader due to missing leader info"));
+      }
       numSnapshotRequests.incrementAndGet();
 
       final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get();