in ratis-server/src/main/java/org/apache/ratis/server/leader/LogAppenderDefault.java [141:179]
public void run() throws InterruptedException, IOException {
while (isRunning()) {
if (shouldSendAppendEntries()) {
SnapshotInfo snapshot = shouldInstallSnapshot();
if (snapshot != null) {
LOG.info("{}: followerNextIndex = {} but logStartIndex = {}, send snapshot {} to follower",
this, getFollower().getNextIndex(), getRaftLog().getStartIndex(), snapshot);
final InstallSnapshotReplyProto r = installSnapshot(snapshot);
if (r != null) {
switch (r.getResult()) {
case NOT_LEADER:
onFollowerTerm(r.getTerm());
break;
case SUCCESS:
case SNAPSHOT_UNAVAILABLE:
case ALREADY_INSTALLED:
case SNAPSHOT_EXPIRED:
getFollower().setAttemptedToInstallSnapshot();
break;
default:
break;
}
}
// otherwise if r is null, retry the snapshot installation
} else {
final AtomicLong requestFirstIndex = new AtomicLong(RaftLog.INVALID_LOG_INDEX);
final AppendEntriesReplyProto r = sendAppendEntriesWithRetries(requestFirstIndex);
if (r != null) {
handleReply(r, requestFirstIndex.get());
}
}
}
if (isRunning() && !hasAppendEntries()) {
getEventAwaitForSignal().await(getHeartbeatWaitTimeMs(), TimeUnit.MILLISECONDS);
}
getLeaderState().checkHealth(getFollower());
}
}