X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FFollower.java;h=4fa0e161db16ffc2ada57fd52cc6af22ce8160fd;hb=2d60632f7cf63712e8357a3cf3fc40d83366e5e6;hp=c35de820dba734d8c48ad9493c372c4cf0d510cb;hpb=b25ae9347455b1bae8f25424a9ceffc017f2f0db;p=controller.git 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 c35de820db..4fa0e161db 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 @@ -20,11 +20,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; import java.io.IOException; import java.util.ArrayList; +import java.util.List; import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import javax.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; @@ -55,6 +57,8 @@ public class Follower extends AbstractRaftActorBehavior { private final SyncStatusTracker initialSyncStatusTracker; + private final MessageAssembler appendEntriesMessageAssembler; + private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted(); private SnapshotTracker snapshotTracker = null; private String leaderId; @@ -73,6 +77,10 @@ public class Follower extends AbstractRaftActorBehavior { initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams() .getSyncIndexThreshold()); + appendEntriesMessageAssembler = MessageAssembler.builder().logContext(logName()) + .fileBackedStreamFactory(context.getFileBackedOutputStreamFactory()) + .assembledMessageCallback((message, sender) -> handleMessage(sender, message)).build(); + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { actor().tell(TimeoutNow.INSTANCE, actor()); } else { @@ -124,18 +132,13 @@ public class Follower extends AbstractRaftActorBehavior { @Override protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) { - - int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; + int numLogEntries = appendEntries.getEntries().size(); if (log.isTraceEnabled()) { log.trace("{}: handleAppendEntries: {}", logName(), appendEntries); } else if (log.isDebugEnabled() && numLogEntries > 0) { log.debug("{}: handleAppendEntries: {}", logName(), appendEntries); } - // 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 && !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()); @@ -157,217 +160,244 @@ public class Follower extends AbstractRaftActorBehavior { leaderId = appendEntries.getLeaderId(); leaderPayloadVersion = appendEntries.getPayloadVersion(); - 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; + } + + if (!processNewEntries(appendEntries, sender)) { + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + return this; + } + long lastIndex = lastIndex(); + long prevCommitIndex = context.getCommitIndex(); - if (isOutOfSync(appendEntries)) { - // We found that the log was out of sync so just send a negative - // reply and return + // If leaderCommit > commitIndex, set commitIndex = min(leaderCommit, index of last new entry) + if (appendEntries.getLeaderCommit() > prevCommitIndex) { + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); + } - final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm(), context.getPayloadVersion()); + if (prevCommitIndex != context.getCommitIndex()) { + log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); + } - log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply); + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm(), context.getPayloadVersion()); - sender.tell(reply, actor()); - return this; + if (log.isTraceEnabled()) { + log.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if (log.isDebugEnabled() && numLogEntries > 0) { + log.debug("{}: handleAppendEntries returning : {}", logName(), reply); } - if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { + // Reply to the leader before applying any previous state so as not to hold up leader consensus. + sender.tell(reply, actor()); - log.debug("{}: Number of entries to be appended = {}", logName(), - appendEntries.getEntries().size()); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); - // 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) { + // 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); + } - // Find the entry up until 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); + applyLogToStateMachine(appendEntries.getLeaderCommit()); + } - if (!isLogEntryPresent(matchEntry.getIndex())) { - // newEntry not found in the log - break; - } + if (!context.getSnapshotManager().isCapturing()) { + super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); + } - long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex()); + appendEntriesMessageAssembler.checkExpiredAssembledMessageState(); - log.debug("{}: matchEntry {} is present: existingEntryTerm: {}", logName(), matchEntry, - existingEntryTerm); + return this; + } - // 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; - } + private boolean processNewEntries(final AppendEntries appendEntries, final ActorRef sender) { + int numLogEntries = appendEntries.getEntries().size(); + if (numLogEntries == 0) { + return true; + } - if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { + log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries); - log.info("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex()); + long lastIndex = lastIndex(); + int addEntriesFrom = 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); + + if (!isLogEntryPresent(matchEntry.getIndex())) { + // newEntry not found in the log + break; + } - // Entries do not match so remove all subsequent entries - if (!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. + long existingEntryTerm = getLogEntryTerm(matchEntry.getIndex()); + + 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.info("{}: Could not remove entries - sending reply to force snapshot", logName()); - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm(), context.getPayloadVersion(), true), actor()); - return this; - } + if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { + log.info("{}: Removing entries from log starting at {}, commitIndex: {}, lastApplied: {}", + logName(), matchEntry.getIndex(), context.getCommitIndex(), context.getLastApplied()); - break; - } else { + // 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), actor()); - return this; + return false; } + + break; + } else { + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion(), true), actor()); + return false; } } + } - 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 Procedure appendAndPersistCallback = logEntry -> { - final ReplicatedLogEntry lastEntryToAppend = appendEntries.getEntries().get( - appendEntries.getEntries().size() - 1); - if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) { - context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex()); - } - }; + 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 Procedure 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()); + } + }; - // 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); + // 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()); + log.debug("{}: Append entry to log {}", logName(), entry.getData()); - context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false); + context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback, false); - shouldCaptureSnapshot.compareAndSet(false, - context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex())); + shouldCaptureSnapshot.compareAndSet(false, + context.getReplicatedLog().shouldCaptureSnapshot(entry.getIndex())); - if (entry.getData() instanceof ServerConfigurationPayload) { - context.updatePeerIds((ServerConfigurationPayload)entry.getData()); - } + if (entry.getData() instanceof ServerConfigurationPayload) { + context.updatePeerIds((ServerConfigurationPayload)entry.getData()); } - - log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); } - // 5. If leaderCommit > commitIndex, set commitIndex = - // min(leaderCommit, index of last new entry) + log.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); - lastIndex = lastIndex(); - long prevCommitIndex = context.getCommitIndex(); + return true; + } - if (appendEntries.getLeaderCommit() > prevCommitIndex) { - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); - } + private boolean isOutOfSync(final AppendEntries appendEntries, final ActorRef sender) { - if (prevCommitIndex != context.getCommitIndex()) { - log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); - } + final long lastIndex = lastIndex(); + if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { - AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex, lastTerm(), context.getPayloadVersion()); + // 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 (log.isTraceEnabled()) { - log.trace("{}: handleAppendEntries returning : {}", logName(), reply); - } else if (log.isDebugEnabled() && numLogEntries > 0) { - log.debug("{}: handleAppendEntries returning : {}", logName(), reply); + log.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(), + appendEntries.getPrevLogIndex()); + + sendOutOfSyncAppendEntriesReply(sender, false); + return true; } - // Reply to the leader before applying any previous state so as not to hold up leader consensus. - sender.tell(reply, actor()); + if (lastIndex > -1) { + if (isLogEntryPresent(appendEntries.getPrevLogIndex())) { + final long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); + if (prevLogTerm != appendEntries.getPrevLogTerm()) { - // 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); - } + // 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 - applyLogToStateMachine(appendEntries.getLeaderCommit()); - } + log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append " + + "entries prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(), + appendEntries.getPrevLogIndex(), prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex, + context.getReplicatedLog().getSnapshotIndex()); - if (!context.getSnapshotManager().isCapturing()) { - super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); + sendOutOfSyncAppendEntriesReply(sender, false); + return true; + } + } else if (appendEntries.getPrevLogIndex() != -1) { + + // 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: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex, + context.getReplicatedLog().getSnapshotIndex()); + + sendOutOfSyncAppendEntriesReply(sender, false); + return true; + } } - return this; - } + 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 - private boolean isOutOfSync(final AppendEntries appendEntries) { + log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the " + + "in-memory journal", logName(), appendEntries.getReplicatedToAllIndex()); - long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); - boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex()); - long lastIndex = lastIndex(); - int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; - boolean outOfSync = true; + sendOutOfSyncAppendEntriesReply(sender, false); + return true; + } - if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) { + 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", logName(), entries.get(0).getIndex() - 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()); - } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) { - - // 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: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex, - context.getReplicatedLog().getSnapshotIndex()); - } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) { - - // 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 - - log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append entries" - + "prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), - prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex, - context.getReplicatedLog().getSnapshotIndex()); - } else if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 - && appendEntries.getReplicatedToAllIndex() != -1 - && !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", logName(), appendEntries.getReplicatedToAllIndex()); - } else if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 - && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0 - && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)) { - log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the " - + " in-memory journal", logName(), appendEntries.getEntries().get(0).getIndex() - 1); - } else { - outOfSync = false; + sendOutOfSyncAppendEntriesReply(sender, false); + return true; + } } - return outOfSync; + + return false; + } + + private void sendOutOfSyncAppendEntriesReply(final ActorRef sender, boolean forceInstallSnapshot) { + // 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); + + log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply); + sender.tell(reply, actor()); } @Override @@ -388,6 +418,10 @@ public class Follower extends AbstractRaftActorBehavior { 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; @@ -588,6 +622,7 @@ public class Follower extends AbstractRaftActorBehavior { public void close() { closeSnapshotTracker(); stopElection(); + appendEntriesMessageAssembler.close(); } @VisibleForTesting