X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FFollower.java;h=288ce32a64ec21286adda9cf3672f7b53d1bf506;hp=0f251a3012e7afe492b867ced86859316fde6d88;hb=d3d04140ac3d7a2a90a3b953cc4ea27fca2bfc32;hpb=919145b1bf7d68e436efa9b22c174965005a174a diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java index 0f251a3012..288ce32a64 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Follower.java @@ -5,29 +5,47 @@ * terms of the Eclipse Public License v1.0 which accompanies this distribution, * and is available at http://www.eclipse.org/legal/epl-v10.html */ - package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.actor.ActorSelection; +import akka.actor.Address; +import akka.cluster.Cluster; +import akka.cluster.ClusterEvent.CurrentClusterState; +import akka.cluster.Member; +import akka.cluster.MemberStatus; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Stopwatch; +import java.io.IOException; import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; +import org.eclipse.jdt.annotation.Nullable; +import org.opendaylight.controller.cluster.messaging.MessageAssembler; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; -import org.opendaylight.controller.cluster.raft.Snapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; -import org.opendaylight.controller.cluster.raft.base.messages.FollowerInitialSyncUpStatus; +import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply; import org.opendaylight.controller.cluster.raft.messages.RaftRPC; +import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; +import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.persisted.Snapshot; /** - * The behavior of a RaftActor in the Follower state - *

+ * The behavior of a RaftActor in the Follower raft state. *

*/ public class Follower extends AbstractRaftActorBehavior { + private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18; + private final SyncStatusTracker initialSyncStatusTracker; + private final MessageAssembler appendEntriesMessageAssembler; + private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted(); private SnapshotTracker snapshotTracker = null; + private String leaderId; + private short leaderPayloadVersion; - private final InitialSyncStatusTracker initialSyncStatusTracker; + public Follower(final RaftActorContext context) { + this(context, null, (short)-1); + } - public Follower(RaftActorContext context) { + public Follower(final RaftActorContext context, final String initialLeaderId, + final short initialLeaderPayloadVersion) { super(context, RaftState.Follower); + this.leaderId = initialLeaderId; + this.leaderPayloadVersion = initialLeaderPayloadVersion; - scheduleElection(electionDuration()); + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams() + .getSyncIndexThreshold()); - initialSyncStatusTracker = new InitialSyncStatusTracker(context.getActor()); - } + appendEntriesMessageAssembler = MessageAssembler.builder().logContext(logName()) + .fileBackedStreamFactory(context.getFileBackedOutputStreamFactory()) + .assembledMessageCallback((message, sender) -> handleMessage(sender, message)).build(); - private boolean isLogEntryPresent(long index){ - if(index == context.getReplicatedLog().getSnapshotIndex()){ - return true; + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(TimeoutNow.INSTANCE, actor()); + } else { + scheduleElection(electionDuration()); } + } - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(index); + @Override + public final String getLeaderId() { + return leaderId; + } - return previousEntry != null; + @VisibleForTesting + protected final void setLeaderId(final @Nullable String leaderId) { + this.leaderId = leaderId; + } + @Override + public short getLeaderPayloadVersion() { + return leaderPayloadVersion; } - private long getLogEntryTerm(long index){ - if(index == context.getReplicatedLog().getSnapshotIndex()){ - return context.getReplicatedLog().getSnapshotTerm(); + @VisibleForTesting + protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) { + this.leaderPayloadVersion = leaderPayloadVersion; + } + + private void restartLastLeaderMessageTimer() { + if (lastLeaderMessageTimer.isRunning()) { + lastLeaderMessageTimer.reset(); } - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(index); + lastLeaderMessageTimer.start(); + } - if(previousEntry != null){ - return previousEntry.getTerm(); + private boolean isLogEntryPresent(final long index) { + if (context.getReplicatedLog().isInSnapshot(index)) { + return true; } - return -1; + ReplicatedLogEntry entry = context.getReplicatedLog().get(index); + return entry != null; + } - private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ - initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); + private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) { + initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex()); } - @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + @Override + protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) { + int numLogEntries = appendEntries.getEntries().size(); + if (log.isTraceEnabled()) { + log.trace("{}: handleAppendEntries: {}", logName(), appendEntries); + } else if (log.isDebugEnabled() && numLogEntries > 0) { + log.debug("{}: handleAppendEntries: {}", logName(), appendEntries); + } - int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; - if(LOG.isTraceEnabled()) { - LOG.trace("{}: handleAppendEntries: {}", logName(), appendEntries); - } else if(LOG.isDebugEnabled() && numLogEntries > 0) { - LOG.debug("{}: handleAppendEntries: {}", logName(), appendEntries); + if (snapshotTracker != null && !snapshotTracker.getLeaderId().equals(appendEntries.getLeaderId())) { + log.debug("{}: snapshot install is in progress but the prior snapshot leaderId {} does not match the " + + "AppendEntries leaderId {}", logName(), snapshotTracker.getLeaderId(), appendEntries.getLeaderId()); + closeSnapshotTracker(); } - // TODO : Refactor this method into a bunch of smaller methods - // to make it easier to read. Before refactoring ensure tests - // cover the code properly + if (snapshotTracker != null || context.getSnapshotManager().isApplying()) { + // if snapshot install is in progress, follower should just acknowledge append entries with a reply. + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex(), lastTerm(), context.getPayloadVersion(), false, needsLeaderAddress(), + appendEntries.getLeaderRaftVersion()); + + log.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply); + sender.tell(reply, actor()); - // 1. Reply false if term < currentTerm (§5.1) - // This is handled in the appendEntries method of the base class + return this; + } // If we got here then we do appear to be talking to the leader leaderId = appendEntries.getLeaderId(); + leaderPayloadVersion = appendEntries.getPayloadVersion(); - // 2. Reply false if log doesn’t contain an entry at prevLogIndex - // whose term matches prevLogTerm (§5.3) + if (appendEntries.getLeaderAddress().isPresent()) { + final String address = appendEntries.getLeaderAddress().get(); + log.debug("New leader address: {}", address); - long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); - boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex()); + context.setPeerAddress(leaderId, address); + context.getConfigParams().getPeerAddressResolver().setResolved(leaderId, address); + } - updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + // First check if the logs are in sync or not + if (isOutOfSync(appendEntries, sender)) { + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + return this; + } - boolean outOfSync = true; + if (!processNewEntries(appendEntries, sender)) { + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + return this; + } - // First check if the logs are in sync or not long lastIndex = lastIndex(); - if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { + long prevCommitIndex = context.getCommitIndex(); + + // If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry) + if (appendEntries.getLeaderCommit() > prevCommitIndex) { + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); + } - // The follower's log is out of sync because the leader does have - // an entry at prevLogIndex and this follower has no entries in - // it's log. + if (prevCommitIndex != context.getCommitIndex()) { + log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); + } - LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}", - logName(), appendEntries.getPrevLogIndex()); - } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) { + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm(), context.getPayloadVersion(), false, needsLeaderAddress(), + appendEntries.getLeaderRaftVersion()); - // The follower's log is out of sync because the Leader's - // prevLogIndex entry was not found in it's log + if (log.isTraceEnabled()) { + log.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if (log.isDebugEnabled() && numLogEntries > 0) { + log.debug("{}: handleAppendEntries returning : {}", logName(), reply); + } - LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it", - logName(), appendEntries.getPrevLogIndex()); - } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) { + // Reply to the leader before applying any previous state so as not to hold up leader consensus. + sender.tell(reply, actor()); - // The follower's log is out of sync because the Leader's - // prevLogIndex entry does exist in the follower's log but it has - // a different term in it + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); - LOG.debug( - "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", - logName(), prevLogTerm, appendEntries.getPrevLogTerm()); - } else { - outOfSync = false; + // If leaderCommit > lastApplied, increment lastApplied and apply log[lastApplied] to state machine (§5.3). + // lastApplied can be equal to lastIndex. + if (appendEntries.getLeaderCommit() > context.getLastApplied() && context.getLastApplied() < lastIndex) { + if (log.isDebugEnabled()) { + log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, " + + "context.getLastApplied(): {}, lastIndex(): {}", logName(), + appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); + } + + applyLogToStateMachine(appendEntries.getLeaderCommit()); } - if (outOfSync) { - // We found that the log was out of sync so just send a negative - // reply and return + if (!context.getSnapshotManager().isCapturing()) { + super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); + } - LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", - logName(), lastIndex, lastTerm()); + appendEntriesMessageAssembler.checkExpiredAssembledMessageState(); - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm()), actor()); - return this; + return this; + } + + private boolean processNewEntries(final AppendEntries appendEntries, final ActorRef sender) { + int numLogEntries = appendEntries.getEntries().size(); + if (numLogEntries == 0) { + return true; } - if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { + log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries); - LOG.debug("{}: Number of entries to be appended = {}", logName(), - appendEntries.getEntries().size()); + long lastIndex = lastIndex(); + int addEntriesFrom = 0; - // 3. If an existing entry conflicts with a new one (same index - // but different terms), delete the existing entry and all that - // follow it (§5.3) - int addEntriesFrom = 0; - if (context.getReplicatedLog().size() > 0) { + // First check for conflicting entries. If an existing entry conflicts with a new one (same index but different + // term), delete the existing entry and all that follow it (§5.3) + if (context.getReplicatedLog().size() > 0) { + // Find the entry up until the one that is not in the follower's log + for (int i = 0;i < numLogEntries; i++, addEntriesFrom++) { + ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i); - // Find the entry up until which the one that is not in the follower's log - for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) { - ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i); - ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex()); + if (!isLogEntryPresent(matchEntry.getIndex())) { + // newEntry not found in the log + break; + } - if (newEntry == null) { - //newEntry not found in the log - break; - } + long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex()); - if (newEntry.getTerm() == matchEntry.getTerm()) { - continue; - } + log.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry, + existingEntryTerm); + + // existingEntryTerm == -1 means it's in the snapshot and not in the log. We don't know + // what the term was so we'll assume it matches. + if (existingEntryTerm == -1 || existingEntryTerm == matchEntry.getTerm()) { + continue; + } - LOG.debug("{}: Removing entries from log starting at {}", logName(), - matchEntry.getIndex()); + if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { + log.info("{}: Removing entries from log starting at {}, commitIndex: {}, lastApplied: {}", + logName(), matchEntry.getIndex(), context.getCommitIndex(), context.getLastApplied()); + + // Entries do not match so remove all subsequent entries but only if the existing entries haven't + // been applied to the state yet. + if (matchEntry.getIndex() <= context.getLastApplied() + || !context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) { + // Could not remove the entries - this means the matchEntry index must be in the + // snapshot and not the log. In this case the prior entries are part of the state + // so we must send back a reply to force a snapshot to completely re-sync the + // follower's log and state. + + log.info("{}: Could not remove entries - sending reply to force snapshot", logName()); + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion(), true, needsLeaderAddress(), + appendEntries.getLeaderRaftVersion()), actor()); + return false; + } - // Entries do not match so remove all subsequent entries - context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex()); break; + } else { + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion(), true, needsLeaderAddress(), + appendEntries.getLeaderRaftVersion()), actor()); + return false; } } + } - lastIndex = lastIndex(); - LOG.debug("{}: After cleanup entries to be added from = {}", logName(), - (addEntriesFrom + lastIndex)); + lastIndex = lastIndex(); + log.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(), lastIndex, + addEntriesFrom); + + // When persistence successfully completes for each new log entry appended, we need to determine if we + // should capture a snapshot to compact the persisted log. shouldCaptureSnapshot tracks whether or not + // one of the log entries has exceeded the log size threshold whereby a snapshot should be taken. However + // we don't initiate the snapshot at that log entry but rather after the last log entry has been persisted. + // This is done because subsequent log entries after the one that tripped the threshold may have been + // applied to the state already, as the persistence callback occurs async, and we want those entries + // purged from the persisted log as well. + final AtomicBoolean shouldCaptureSnapshot = new AtomicBoolean(false); + final Consumer appendAndPersistCallback = logEntry -> { + final List entries = appendEntries.getEntries(); + final ReplicatedLogEntry lastEntryToAppend = entries.get(entries.size() - 1); + if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) { + context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex()); + } + }; + + // Append any new entries not already in the log + for (int i = addEntriesFrom; i < numLogEntries; i++) { + ReplicatedLogEntry entry = appendEntries.getEntries().get(i); + + log.debug("{}: Append entry to log {}", logName(), entry.getData()); - // 4. Append any new entries not already in the log - for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { - ReplicatedLogEntry entry = appendEntries.getEntries().get(i); + context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false); - LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); + shouldCaptureSnapshot.compareAndSet(false, + context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex())); - context.getReplicatedLog().appendAndPersist(entry); + if (entry.getData() instanceof ServerConfigurationPayload) { + context.updatePeerIds((ServerConfigurationPayload)entry.getData()); } + } + + log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); - LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); + return true; + } + + private boolean isOutOfSync(final AppendEntries appendEntries, final ActorRef sender) { + + final long lastIndex = lastIndex(); + if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { + + // The follower's log is out of sync because the leader does have an entry at prevLogIndex and this + // follower has no entries in it's log. + + log.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(), + appendEntries.getPrevLogIndex()); + + sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion()); + return true; } - // 5. If leaderCommit > commitIndex, set commitIndex = - // min(leaderCommit, index of last new entry) + if (lastIndex > -1) { + if (isLogEntryPresent(appendEntries.getPrevLogIndex())) { + final long leadersPrevLogTermInFollowersLogOrSnapshot = + getLogEntryOrSnapshotTerm(appendEntries.getPrevLogIndex()); + if (leadersPrevLogTermInFollowersLogOrSnapshot != appendEntries.getPrevLogTerm()) { - lastIndex = lastIndex(); - long prevCommitIndex = context.getCommitIndex(); + // The follower's log is out of sync because the Leader's prevLogIndex entry does exist + // in the follower's log or snapshot but it has a different term. - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); + log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append " + + "entries prevLogTerm {} - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(), + appendEntries.getPrevLogIndex(), leadersPrevLogTermInFollowersLogOrSnapshot, + appendEntries.getPrevLogTerm(), lastIndex, context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm()); - if (prevCommitIndex != context.getCommitIndex()) { - LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); - } - - // If commitIndex > lastApplied: increment lastApplied, apply - // log[lastApplied] to state machine (§5.3) - // check if there are any entries to be applied. last-applied can be equal to last-index - if (appendEntries.getLeaderCommit() > context.getLastApplied() && - context.getLastApplied() < lastIndex) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: applyLogToStateMachine, " + - "appendEntries.getLeaderCommit(): {}," + - "context.getLastApplied(): {}, lastIndex(): {}", logName(), - appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); - } + sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion()); + return true; + } + } else if (appendEntries.getPrevLogIndex() != -1) { - applyLogToStateMachine(appendEntries.getLeaderCommit()); + // The follower's log is out of sync because the Leader's prevLogIndex entry was not found in it's log + + log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, " + + "snapshotIndex: {}, snapshotTerm: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex, + context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm()); + + sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion()); + return true; + } } - AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex, lastTerm()); + if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 + && appendEntries.getReplicatedToAllIndex() != -1) { + if (!isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) { + // This append entry comes from a leader who has it's log aggressively trimmed and so does not have + // the previous entry in it's in-memory journal + + log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the " + + "in-memory journal - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(), + appendEntries.getReplicatedToAllIndex(), lastIndex, + context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm()); + + sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion()); + return true; + } - if(LOG.isTraceEnabled()) { - LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply); - } else if(LOG.isDebugEnabled() && numLogEntries > 0) { - LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply); + final List entries = appendEntries.getEntries(); + if (entries.size() > 0 && !isLogEntryPresent(entries.get(0).getIndex() - 1)) { + log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the " + + "in-memory journal - lastIndex: {}, snapshotIndex: {}, snapshotTerm: {}", logName(), + entries.get(0).getIndex() - 1, lastIndex, context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm()); + + sendOutOfSyncAppendEntriesReply(sender, false, appendEntries.getLeaderRaftVersion()); + return true; + } } + return false; + } + + private void sendOutOfSyncAppendEntriesReply(final ActorRef sender, final boolean forceInstallSnapshot, + final short leaderRaftVersion) { + // We found that the log was out of sync so just send a negative reply. + final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(), + lastTerm(), context.getPayloadVersion(), forceInstallSnapshot, needsLeaderAddress(), + leaderRaftVersion); + + log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply); sender.tell(reply, actor()); + } - if (!context.isSnapshotCaptureInitiated()) { - super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); - } + private boolean needsLeaderAddress() { + return context.getPeerAddress(leaderId) == null; + } + @Override + protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender, + final AppendEntriesReply appendEntriesReply) { return this; } - @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply) { + @Override + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, + final RequestVoteReply requestVoteReply) { return this; } - @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + @Override + final ApplyState getApplyStateFor(final ReplicatedLogEntry entry) { + return new ApplyState(null, null, entry); + } + + @Override + public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { + if (message instanceof ElectionTimeout || message instanceof TimeoutNow) { + return handleElectionTimeout(message); + } + + if (appendEntriesMessageAssembler.handleMessage(message, actor())) { + return this; + } + + if (!(message instanceof RaftRPC)) { + // The rest of the processing requires the message to be a RaftRPC + return null; + } + + final RaftRPC rpc = (RaftRPC) message; + // If RPC request or response contains term T > currentTerm: + // set currentTerm = T, convert to follower (§5.1) + // This applies to all RPC messages and responses + if (rpc.getTerm() > context.getTermInformation().getCurrentTerm() && shouldUpdateTerm(rpc)) { + log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); + } + + if (rpc instanceof InstallSnapshot) { + handleInstallSnapshot(sender, (InstallSnapshot) rpc); + restartLastLeaderMessageTimer(); + scheduleElection(electionDuration()); + return this; + } + + if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) { + restartLastLeaderMessageTimer(); + scheduleElection(electionDuration()); + } + + return super.handleMessage(sender, rpc); + } + + private RaftActorBehavior handleElectionTimeout(final Object message) { + // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader + // during the election timeout interval. It may that the election timer expired b/c this actor + // was busy and messages got delayed, in which case leader messages would be backed up in the + // queue but would be processed before the ElectionTimeout message and thus would restart the + // lastLeaderMessageTimer. + long lastLeaderMessageInterval = lastLeaderMessageTimer.elapsed(TimeUnit.MILLISECONDS); + long electionTimeoutInMillis = context.getConfigParams().getElectionTimeOutInterval().toMillis(); + boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning() + || lastLeaderMessageInterval >= electionTimeoutInMillis; + + if (canStartElection()) { + if (message instanceof TimeoutNow) { + log.debug("{}: Received TimeoutNow - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } else if (noLeaderMessageReceived) { + // Check the cluster state to see if the leader is known to be up before we go to Candidate. + // However if we haven't heard from the leader in a long time even though the cluster state + // indicates it's up then something is wrong - leader might be stuck indefinitely - so switch + // to Candidate, + long maxElectionTimeout = electionTimeoutInMillis * MAX_ELECTION_TIMEOUT_FACTOR; + if (isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) { + log.debug("{}: Received ElectionTimeout but leader appears to be available", logName()); + scheduleElection(electionDuration()); + } else if (isThisFollowerIsolated()) { + log.debug("{}: this follower is isolated. Do not switch to Candidate for now.", logName()); + setLeaderId(null); + scheduleElection(electionDuration()); + } else { + log.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } + } else { + log.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout {}", + logName(), lastLeaderMessageInterval, context.getConfigParams().getElectionTimeOutInterval()); + scheduleElection(electionDuration()); + } + } else if (message instanceof ElectionTimeout) { + if (noLeaderMessageReceived) { + setLeaderId(null); + } + + scheduleElection(electionDuration()); + } + return this; } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + private boolean isLeaderAvailabilityKnown() { + if (leaderId == null) { + return false; + } + + Optional cluster = context.getCluster(); + if (!cluster.isPresent()) { + return false; + } + + ActorSelection leaderActor = context.getPeerActorSelection(leaderId); + if (leaderActor == null) { + return false; + } + + Address leaderAddress = leaderActor.anchorPath().address(); + + CurrentClusterState state = cluster.get().state(); + Set unreachable = state.getUnreachable(); - Object message = fromSerializableMessage(originalMessage); + log.debug("{}: Checking for leader {} in the cluster unreachable set {}", logName(), leaderAddress, + unreachable); - if (message instanceof RaftRPC) { - RaftRPC rpc = (RaftRPC) message; - // If RPC request or response contains term T > currentTerm: - // set currentTerm = T, convert to follower (§5.1) - // This applies to all RPC messages and responses - if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { - LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", - logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + for (Member m: unreachable) { + if (leaderAddress.equals(m.address())) { + log.info("{}: Leader {} is unreachable", logName(), leaderAddress); + return false; + } + } - context.getTermInformation().updateAndPersist(rpc.getTerm(), null); + for (Member m: state.getMembers()) { + if (leaderAddress.equals(m.address())) { + if (m.status() == MemberStatus.up() || m.status() == MemberStatus.weaklyUp()) { + log.debug("{}: Leader {} cluster status is {} - leader is available", logName(), + leaderAddress, m.status()); + return true; + } else { + log.debug("{}: Leader {} cluster status is {} - leader is unavailable", logName(), + leaderAddress, m.status()); + return false; + } } } - if (message instanceof ElectionTimeout) { - LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); - return switchBehavior(new Candidate(context)); + log.debug("{}: Leader {} not found in the cluster member set", logName(), leaderAddress); + + return false; + } + + private boolean isThisFollowerIsolated() { + final Optional maybeCluster = context.getCluster(); + if (!maybeCluster.isPresent()) { + return false; + } + + final Cluster cluster = maybeCluster.get(); + final Member selfMember = cluster.selfMember(); + + final CurrentClusterState state = cluster.state(); + final Set unreachable = state.getUnreachable(); + final Iterable members = state.getMembers(); - } else if (message instanceof InstallSnapshot) { - InstallSnapshot installSnapshot = (InstallSnapshot) message; - handleInstallSnapshot(sender, installSnapshot); + log.debug("{}: Checking if this node is isolated in the cluster unreachable set {}," + + "all members {} self member: {}", logName(), unreachable, members, selfMember); + + // no unreachable peers means we cannot be isolated + if (unreachable.size() == 0) { + return false; } - scheduleElection(electionDuration()); + final Set membersToCheck = new HashSet<>(); + members.forEach(membersToCheck::add); + + membersToCheck.removeAll(unreachable); - return super.handleMessage(sender, message); + // check if the only member not unreachable is us + if (membersToCheck.size() == 1 && membersToCheck.iterator().next().equals(selfMember)) { + return true; + } + + return false; } - private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) { + private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) { + + log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); - LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}", - logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(), - installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks()); + leaderId = installSnapshot.getLeaderId(); - if(snapshotTracker == null){ - snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); + if (snapshotTracker == null) { + snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(), + context); } updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); try { - if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), - installSnapshot.getLastChunkHashCode())){ - Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(), - new ArrayList(), + final InstallSnapshotReply reply = new InstallSnapshotReply( + currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); + + if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), + installSnapshot.getLastChunkHashCode())) { + + log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId()); + + Snapshot snapshot = Snapshot.create( + context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()), + new ArrayList<>(), installSnapshot.getLastIncludedIndex(), installSnapshot.getLastIncludedTerm(), installSnapshot.getLastIncludedIndex(), - installSnapshot.getLastIncludedTerm()); - - actor().tell(new ApplySnapshot(snapshot), actor()); + installSnapshot.getLastIncludedTerm(), + context.getTermInformation().getCurrentTerm(), + context.getTermInformation().getVotedFor(), + installSnapshot.getServerConfig().orElse(null)); - snapshotTracker = null; + ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() { + @Override + public void onSuccess() { + log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); - } + sender.tell(reply, actor()); + } - InstallSnapshotReply reply = new InstallSnapshotReply( - currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); + @Override + public void onFailure() { + sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor()); + } + }; - LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor()); - sender.tell(reply, actor()); + closeSnapshotTracker(); + } else { + log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); - } catch (SnapshotTracker.InvalidChunkException e) { - LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e); + sender.tell(reply, actor()); + } + } catch (IOException e) { + log.debug("{}: Exception in InstallSnapshot of follower", logName(), e); sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor()); - snapshotTracker = null; - - } catch (Exception e){ - LOG.error("{}: Exception in InstallSnapshot of follower", logName(), e); - //send reply with success as false. The chunk will be sent again on failure - sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), - installSnapshot.getChunkIndex(), false), actor()); + closeSnapshotTracker(); + } + } + private void closeSnapshotTracker() { + if (snapshotTracker != null) { + snapshotTracker.close(); + snapshotTracker = null; } } @Override - public void close() throws Exception { + public void close() { + closeSnapshotTracker(); stopElection(); + appendEntriesMessageAssembler.close(); } @VisibleForTesting - SnapshotTracker getSnapshotTracker(){ + SnapshotTracker getSnapshotTracker() { return snapshotTracker; } - - private class InitialSyncStatusTracker { - - private static final long INVALID_LOG_INDEX = -2L; - private long initialLeaderCommit = INVALID_LOG_INDEX; - private boolean initialSyncUpDone = false; - private String syncedLeaderId = null; - private final ActorRef actor; - - public InitialSyncStatusTracker(ActorRef actor) { - this.actor = actor; - } - - public void update(String leaderId, long leaderCommit, long commitIndex){ - - if(!leaderId.equals(syncedLeaderId)){ - initialSyncUpDone = false; - initialLeaderCommit = INVALID_LOG_INDEX; - syncedLeaderId = leaderId; - } - - if(!initialSyncUpDone){ - if(initialLeaderCommit == INVALID_LOG_INDEX){ - actor.tell(new FollowerInitialSyncUpStatus(false, getId()), ActorRef.noSender()); - initialLeaderCommit = leaderCommit; - } else if(commitIndex >= initialLeaderCommit){ - actor.tell(new FollowerInitialSyncUpStatus(true, getId()), ActorRef.noSender()); - initialSyncUpDone = true; - } - } - } - } }