in ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java [679:742]
public void onNext(InstallSnapshotReplyProto reply) {
if (LOG.isInfoEnabled()) {
LOG.info("{}: received {} reply {}", this, replyState.isFirstReplyReceived()? "a" : "the first",
ServerStringUtils.toInstallSnapshotReplyString(reply));
}
// update the last rpc time
getFollower().updateLastRpcResponseTime();
replyState.process(Event.SNAPSHOT_REPLY);
final long followerSnapshotIndex;
switch (reply.getResult()) {
case SUCCESS:
LOG.info("{}: Completed InstallSnapshot. Reply: {}", this, reply);
getFollower().setAttemptedToInstallSnapshot();
removePending(reply);
break;
case IN_PROGRESS:
LOG.info("{}: InstallSnapshot in progress.", this);
removePending(reply);
break;
case ALREADY_INSTALLED:
followerSnapshotIndex = reply.getSnapshotIndex();
LOG.info("{}: Follower snapshot is already at index {}.", this, followerSnapshotIndex);
getFollower().setSnapshotIndex(followerSnapshotIndex);
getFollower().setAttemptedToInstallSnapshot();
getLeaderState().onFollowerCommitIndex(getFollower(), followerSnapshotIndex);
increaseNextIndex(followerSnapshotIndex, reply.getResult());
removePending(reply);
break;
case NOT_LEADER:
onFollowerTerm(reply.getTerm());
break;
case CONF_MISMATCH:
LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but follower {} has it set to {}",
this, RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY,
getServer().getId(), installSnapshotEnabled, getFollowerId(), !installSnapshotEnabled);
break;
case SNAPSHOT_INSTALLED:
followerSnapshotIndex = reply.getSnapshotIndex();
LOG.info("{}: Follower installed snapshot at index {}", this, followerSnapshotIndex);
getFollower().setSnapshotIndex(followerSnapshotIndex);
getFollower().setAttemptedToInstallSnapshot();
getLeaderState().onFollowerCommitIndex(getFollower(), followerSnapshotIndex);
increaseNextIndex(followerSnapshotIndex, reply.getResult());
onFollowerCatchup(followerSnapshotIndex);
removePending(reply);
break;
case SNAPSHOT_UNAVAILABLE:
LOG.info("{}: Follower could not install snapshot as it is not available.", this);
getFollower().setAttemptedToInstallSnapshot();
notifyInstallSnapshotFinished(InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, RaftLog.INVALID_LOG_INDEX);
removePending(reply);
break;
case UNRECOGNIZED:
LOG.error("Unrecognized the reply result {}: Leader is {}, follower is {}",
reply.getResult(), getServer().getId(), getFollowerId());
break;
case SNAPSHOT_EXPIRED:
LOG.warn("{}: Follower could not install snapshot as it is expired.", this);
default:
break;
}
}