[ 
https://issues.apache.org/jira/browse/RATIS-1326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17294285#comment-17294285
 ] 

Shashikant Banerjee commented on RATIS-1326:
--------------------------------------------

The issue here is as follows:
{code:java}
RaftServerImpl.java:

RoleInfoProto getRoleInfoProto() {
  RaftPeerRole currentRole = role.getCurrentRole();
  RoleInfoProto.Builder roleInfo = RoleInfoProto.newBuilder()
      .setSelf(getPeer().getRaftPeerProto())
      .setRole(currentRole)
      .setRoleElapsedTimeMs(role.getRoleElapsedTimeMs());
  switch (currentRole) {
  case CANDIDATE:
    CandidateInfoProto.Builder candidate = CandidateInfoProto.newBuilder()
        .setLastLeaderElapsedTimeMs(state.getLastLeaderElapsedTimeMs());
    roleInfo.setCandidateInfo(candidate);
    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()));
    });
    break;

{code}
While building the roleInfo proto during the installSnapshot call during 
setConfiguration(), the follower state is null as the node is not yet a voting 
member. As a result, the leader info is not set.

> NotifyInstallSnapshot during SetConfiguration has  leader info missing
> ----------------------------------------------------------------------
>
>                 Key: RATIS-1326
>                 URL: https://issues.apache.org/jira/browse/RATIS-1326
>             Project: Ratis
>          Issue Type: Bug
>          Components: server
>            Reporter: Shashikant Banerjee
>            Assignee: Shashikant Banerjee
>            Priority: Major
>
> Sample patch to trigger the failure:
> {code:java}
> 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();
> {code}
> {code:java}
> 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)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to