Uploaded image for project: 'Ratis'
  1. Ratis
  2. RATIS-1326

NotifyInstallSnapshot during SetConfiguration has leader info missing

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 2.1.0
    • server
    • None

    Description

      Sample patch to trigger the failure:

      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 f932fbb7..8039b6b5 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
      @@ -1569,6 +1569,7 @@ class RaftServerImpl implements RaftServer.Division,
                     });
               } catch (Throwable t) {
                 inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogTermIndex, null);
      +          LOG.info("InstallSnapshotFromLeader Failed", t);
                 throw t;
               }
       
      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 a4c25da4..899f5c07 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,9 @@ public abstract class InstallSnapshotNotificationTests<CLUSTER extends MiniRaftC
           public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader(
               RaftProtos.RoleInfoProto roleInfoProto,
               TermIndex termIndex) {
      -
      +      if (roleInfoProto.getFollowerInfo().getLeaderInfo().getId().getId().isEmpty()) {
      +        Assert.fail();
      +      }
             numSnapshotRequests.incrementAndGet();
       
             final SingleFileSnapshotInfo leaderSnapshotInfo = (SingleFileSnapshotInfo) leaderSnapshotInfoRef.get();
      
      
      2021-03-03 10:39:49,176 [grpc-default-executor-4] INFO  server.RaftServer$Division (RaftServerImpl.java:notifyStateMachineToInstallSnapshot(1572)) - InstallSnapshotFromLeader Failed2021-03-03 10:39:49,176 [grpc-default-executor-4] INFO  server.RaftServer$Division (RaftServerImpl.java:notifyStateMachineToInstallSnapshot(1572)) - InstallSnapshotFromLeader Failedjava.lang.AssertionError at org.junit.Assert.fail(Assert.java:86) at org.junit.Assert.fail(Assert.java:95) at org.apache.ratis.InstallSnapshotNotificationTests$StateMachine4InstallSnapshotNotificationTests.notifyInstallSnapshotFromLeader(InstallSnapshotNotificationTests.java:90) at org.apache.ratis.server.impl.RaftServerImpl.notifyStateMachineToInstallSnapshot(RaftServerImpl.java:1552) at org.apache.ratis.server.impl.RaftServerImpl.installSnapshotImpl(RaftServerImpl.java:1432) at org.apache.ratis.server.impl.RaftServerImpl.installSnapshot(RaftServerImpl.java:1316) at org.apache.ratis.server.impl.RaftServerProxy.installSnapshot(RaftServerProxy.java:569) at org.apache.ratis.grpc.server.GrpcServerProtocolService$2.process(GrpcServerProtocolService.java:239) at org.apache.ratis.grpc.server.GrpcServerProtocolService$2.process(GrpcServerProtocolService.java:236) at org.apache.ratis.grpc.server.GrpcServerProtocolService$ServerRequestStreamObserver.onNext(GrpcServerProtocolService.java:126) at org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:255) at org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:309) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:292) at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:782) at org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37) at org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748)
      

      Attachments

        Issue Links

          Activity

            People

              shashikant Shashikant Banerjee
              shashikant Shashikant Banerjee
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 40m
                  40m