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=410b3c266c87066cef3828732f62ca154a580df3;hp=68b444b2bac7d20ae0ab7a307ede546191392938;hb=f276ae33b951d173b51c467bb7bb1a5f5cf9a1e6;hpb=d097d70beab05f7fcb028ff12b2c47e35570fb55 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 68b444b2ba..7e6654d5d6 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 @@ -9,15 +9,23 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.japi.Procedure; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.util.ArrayList; 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.internal.messages.ApplySnapshot; -import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout; +import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.Snapshot; +import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout; 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; /** @@ -31,89 +39,143 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; * */ public class Follower extends AbstractRaftActorBehavior { + private static final int SYNC_THRESHOLD = 10; + + private final SyncStatusTracker initialSyncStatusTracker; + + private final Procedure appendAndPersistCallback = new Procedure() { + @Override + public void apply(ReplicatedLogEntry logEntry) { + context.getReplicatedLog().captureSnapshotIfReady(logEntry); + } + }; + + private SnapshotTracker snapshotTracker = null; + private String leaderId; + private short leaderPayloadVersion; + public Follower(RaftActorContext context) { - super(context); + this(context, null, (short)-1); + } + + public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) { + super(context, RaftState.Follower); + this.leaderId = initialLeaderId; + this.leaderPayloadVersion = initialLeaderPayloadVersion; + + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); - scheduleElection(electionDuration()); + if(canStartElection()) { + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(ElectionTimeout.INSTANCE, actor()); + } else { + scheduleElection(electionDuration()); + } + } } - @Override protected RaftState handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + @Override + public final String getLeaderId() { + return leaderId; + } - // TODO : Refactor this method into a bunch of smaller methods - // to make it easier to read. Before refactoring ensure tests - // cover the code properly + @VisibleForTesting + protected final void setLeaderId(final String leaderId) { + this.leaderId = Preconditions.checkNotNull(leaderId); + } - // 1. Reply false if term < currentTerm (§5.1) - // This is handled in the appendEntries method of the base class + @Override + public short getLeaderPayloadVersion() { + return leaderPayloadVersion; + } - // If we got here then we do appear to be talking to the leader - leaderId = appendEntries.getLeaderId(); + @VisibleForTesting + protected final void setLeaderPayloadVersion(short leaderPayloadVersion) { + this.leaderPayloadVersion = leaderPayloadVersion; + } - // 2. Reply false if log doesn’t contain an entry at prevLogIndex - // whose term matches prevLogTerm (§5.3) + private boolean isLogEntryPresent(long index){ + if(index == context.getReplicatedLog().getSnapshotIndex()){ + return true; + } ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(appendEntries.getPrevLogIndex()); + .get(index); + return previousEntry != null; - boolean outOfSync = true; + } - // First check if the logs are in sync or not - if (lastIndex() == -1 - && appendEntries.getPrevLogIndex() != -1) { + private long getLogEntryTerm(long index){ + if(index == context.getReplicatedLog().getSnapshotIndex()){ + return context.getReplicatedLog().getSnapshotTerm(); + } - // 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. + ReplicatedLogEntry previousEntry = context.getReplicatedLog() + .get(index); + + if(previousEntry != null){ + return previousEntry.getTerm(); + } - context.getLogger().debug( - "The followers log is empty and the senders prevLogIndex is {}", - appendEntries.getPrevLogIndex()); + return -1; + } - } else if (lastIndex() > -1 - && appendEntries.getPrevLogIndex() != -1 - && previousEntry == null) { + private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ + initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); + } - // The follower's log is out of sync because the Leader's - // prevLogIndex entry was not found in it's log + @Override + protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { - context.getLogger().debug( - "The log is not empty but the prevLogIndex {} was not found in it", - appendEntries.getPrevLogIndex()); + 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); + } - } else if (lastIndex() > -1 - && previousEntry != null - && previousEntry.getTerm()!= appendEntries.getPrevLogTerm()) { + // TODO : Refactor this method into a bunch of smaller methods + // to make it easier to read. Before refactoring ensure tests + // cover the code properly - // 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 + 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()); - context.getLogger().debug( - "Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}" - , previousEntry.getTerm() - , appendEntries.getPrevLogTerm()); - } else { - outOfSync = false; + if(LOG.isDebugEnabled()) { + LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply); + } + sender.tell(reply, actor()); + + return this; } - if (outOfSync) { + // If we got here then we do appear to be talking to the leader + leaderId = appendEntries.getLeaderId(); + leaderPayloadVersion = appendEntries.getPayloadVersion(); + + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + // First check if the logs are in sync or not + long lastIndex = lastIndex(); + + if (isOutOfSync(appendEntries)) { // We found that the log was out of sync so just send a negative // reply and return - sender.tell( - new AppendEntriesReply(context.getId(), currentTerm(), false, - lastIndex(), lastTerm()), actor() - ); - return state(); + + LOG.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", + logName(), lastIndex, lastTerm()); + + sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion()), actor()); + return this; } - if (appendEntries.getEntries() != null - && appendEntries.getEntries().size() > 0) { - context.getLogger().debug( - "Number of entries to be appended = " + appendEntries - .getEntries().size() - ); + if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) { + + LOG.debug("{}: Number of entries to be appended = {}", logName(), + appendEntries.getEntries().size()); // 3. If an existing entry conflicts with a new one (same index // but different terms), delete the existing entry and all that @@ -121,127 +183,284 @@ public class Follower extends AbstractRaftActorBehavior { int addEntriesFrom = 0; if (context.getReplicatedLog().size() > 0) { - // 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()); + // 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); + ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex()); if (newEntry == null) { //newEntry not found in the log break; } - if (newEntry.getTerm() == matchEntry - .getTerm()) { + if (newEntry.getTerm() == matchEntry.getTerm()) { continue; } - context.getLogger().debug( - "Removing entries from log starting at " - + matchEntry.getIndex() - ); + if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { - // Entries do not match so remove all subsequent entries - context.getReplicatedLog() - .removeFromAndPersist(matchEntry.getIndex()); - break; + LOG.debug("{}: Removing entries from log starting at {}", logName(), + matchEntry.getIndex()); + + // 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), actor()); + return this; + } } } - context.getLogger().debug( - "After cleanup entries to be added from = " + (addEntriesFrom - + lastIndex()) - ); + lastIndex = lastIndex(); + LOG.debug("{}: After cleanup entries to be added from = {}", logName(), + (addEntriesFrom + lastIndex)); // 4. Append any new entries not already in the log - for (int i = addEntriesFrom; - i < appendEntries.getEntries().size(); i++) { - - context.getLogger().debug( - "Append entry to log " + appendEntries.getEntries().get(i).getData() - .toString() - ); - context.getReplicatedLog() - .appendAndPersist(appendEntries.getEntries().get(i)); + for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { + ReplicatedLogEntry entry = appendEntries.getEntries().get(i); + + LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); + + context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback); + + if(entry.getData() instanceof ServerConfigurationPayload) { + context.updatePeerIds((ServerConfigurationPayload)entry.getData()); + } } - context.getLogger().debug( - "Log size is now " + context.getReplicatedLog().size()); + LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); } - // 5. If leaderCommit > commitIndex, set commitIndex = // min(leaderCommit, index of last new entry) + lastIndex = lastIndex(); long prevCommitIndex = context.getCommitIndex(); - context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), - context.getReplicatedLog().lastIndex())); + context.setCommitIndex(Math.min(appendEntries.getLeaderCommit(), lastIndex)); if (prevCommitIndex != context.getCommitIndex()) { - context.getLogger() - .debug("Commit index set to " + context.getCommitIndex()); + LOG.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); } // If commitIndex > lastApplied: increment lastApplied, apply // log[lastApplied] to state machine (§5.3) - if (appendEntries.getLeaderCommit() > context.getLastApplied()) { + // 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); + } + applyLogToStateMachine(appendEntries.getLeaderCommit()); } - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex(), lastTerm()), actor()); + AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, + lastIndex, lastTerm(), context.getPayloadVersion()); + + if(LOG.isTraceEnabled()) { + LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply); + } else if(LOG.isDebugEnabled() && numLogEntries > 0) { + LOG.debug("{}: handleAppendEntries returning : {}", logName(), reply); + } + + sender.tell(reply, actor()); + + if (!context.getSnapshotManager().isCapturing()) { + super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex()); + } + + return this; + } + + private boolean isOutOfSync(AppendEntries appendEntries) { + + 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; + + 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.debug("{}: 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.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()) { + + // 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 - return state(); + LOG.debug( + "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", + logName(), prevLogTerm, appendEntries.getPrevLogTerm()); + } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 + && appendEntries.getReplicatedToAllIndex() != -1 + && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex()) + && !context.getReplicatedLog().isInSnapshot(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.debug( + "{}: 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) + && !context.getReplicatedLog().isInSnapshot(appendEntries.getEntries().get(0).getIndex() - 1)) { + LOG.debug( + "{}: 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; + } + return outOfSync; } - @Override protected RaftState handleAppendEntriesReply(ActorRef sender, + @Override + protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { - return state(); + return this; } - @Override protected RaftState handleRequestVoteReply(ActorRef sender, + @Override + protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { - return state(); + return this; } - @Override public RaftState state() { - return RaftState.Follower; - } + @Override + public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + if (originalMessage instanceof ElectionTimeout) { + if (canStartElection()) { + LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } else { + return this; + } + } - @Override public RaftState handleMessage(ActorRef sender, Object originalMessage) { + final Object message = fromSerializableMessage(originalMessage); + if (!(message instanceof RaftRPC)) { + // The rest of the processing requires the message to be a RaftRPC + return null; + } - Object message = fromSerializableMessage(originalMessage); + 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()) { + LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); - 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()) { - context.getTermInformation().updateAndPersist(rpc.getTerm(), null); - } + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); } - if (message instanceof ElectionTimeout) { - return RaftState.Candidate; + if (rpc instanceof InstallSnapshot) { + InstallSnapshot installSnapshot = (InstallSnapshot) rpc; + handleInstallSnapshot(sender, installSnapshot); + scheduleElection(electionDuration()); + return this; + } - } else if (message instanceof InstallSnapshot) { - InstallSnapshot installSnapshot = (InstallSnapshot) message; - actor().tell(new ApplySnapshot(installSnapshot.getData()), actor()); + if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) { + scheduleElection(electionDuration()); } - scheduleElection(electionDuration()); + return super.handleMessage(sender, rpc); + } + + private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { + + LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); + + leaderId = installSnapshot.getLeaderId(); + + if(snapshotTracker == null){ + snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); + } - return super.handleMessage(sender, message); + updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); + + try { + final InstallSnapshotReply reply = new InstallSnapshotReply( + currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true); + + if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), + installSnapshot.getLastChunkHashCode())){ + Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(), + new ArrayList(), + installSnapshot.getLastIncludedIndex(), + installSnapshot.getLastIncludedTerm(), + installSnapshot.getLastIncludedIndex(), + installSnapshot.getLastIncludedTerm(), + context.getTermInformation().getCurrentTerm(), + context.getTermInformation().getVotedFor(), + context.getPeerServerInfo(true)); + + ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() { + @Override + public void onSuccess() { + LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + + sender.tell(reply, actor()); + } + + @Override + public void onFailure() { + sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor()); + } + }; + + actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor()); + + snapshotTracker = null; + } else { + LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); + + sender.tell(reply, actor()); + } + } catch (SnapshotTracker.InvalidChunkException 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()); + + } } - @Override public void close() throws Exception { + @Override + public void close() { stopElection(); } + + @VisibleForTesting + SnapshotTracker getSnapshotTracker(){ + return snapshotTracker; + } }