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=0149b57e6e94f8ddf076a406f14eea1546d2aa1e;hp=1e4fcf7225daf60e0078a80ec0ca5403e0c79eab;hb=8882e6077db69d22bcc57fcf12dd4a02a81a4967;hpb=6dfafe7c19c5e79bce2cb8aa2470b23360c08147 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 1e4fcf7225..0149b57e6e 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 @@ -10,11 +10,11 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; 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.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; @@ -23,6 +23,7 @@ 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; /** @@ -39,10 +40,28 @@ public class Follower extends AbstractRaftActorBehavior { private SnapshotTracker snapshotTracker = null; + private final SyncStatusTracker initialSyncStatusTracker; + + private static final int SYNC_THRESHOLD = 10; + public Follower(RaftActorContext context) { + this(context, null); + } + + public Follower(RaftActorContext context, String initialLeaderId) { super(context, RaftState.Follower); + leaderId = initialLeaderId; + + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); + + if(context.getRaftPolicy().automaticElectionsEnabled()) { + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(ELECTION_TIMEOUT, actor()); + } else { + scheduleElection(electionDuration()); + } + } - scheduleElection(electionDuration()); } private boolean isLogEntryPresent(long index){ @@ -72,6 +91,10 @@ public class Follower extends AbstractRaftActorBehavior { return -1; } + private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ + initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); + } + @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { @@ -86,52 +109,29 @@ public class Follower extends AbstractRaftActorBehavior { // to make it easier to read. Before refactoring ensure tests // cover the code properly - // 1. Reply false if term < currentTerm (§5.1) - // This is handled in the appendEntries method of the base class + 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()); - // If we got here then we do appear to be talking to the leader - leaderId = appendEntries.getLeaderId(); - - // 2. Reply false if log doesn’t contain an entry at prevLogIndex - // whose term matches prevLogTerm (§5.3) + if(LOG.isDebugEnabled()) { + LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply); + } + sender.tell(reply, actor()); - long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); - boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex()); + return this; + } + // If we got here then we do appear to be talking to the leader + leaderId = appendEntries.getLeaderId(); - boolean outOfSync = true; + setLeaderPayloadVersion(appendEntries.getPayloadVersion()); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); // First check if the logs are in sync or not 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.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 - - LOG.debug( - "{}: Cannot append entries because previous entry term {} is not equal to append entries prevLogTerm {}", - logName(), prevLogTerm, appendEntries.getPrevLogTerm()); - } else { - outOfSync = false; - } - - if (outOfSync) { + if (isOutOfSync(appendEntries)) { // We found that the log was out of sync so just send a negative // reply and return @@ -139,7 +139,7 @@ public class Follower extends AbstractRaftActorBehavior { logName(), lastIndex, lastTerm()); sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm()), actor()); + lastTerm(), context.getPayloadVersion()), actor()); return this; } @@ -154,7 +154,7 @@ 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 + // 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()); @@ -168,12 +168,19 @@ public class Follower extends AbstractRaftActorBehavior { continue; } - LOG.debug("{}: Removing entries from log starting at {}", logName(), + if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { + + 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; + // 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; + } } } @@ -188,6 +195,10 @@ public class Follower extends AbstractRaftActorBehavior { LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); context.getReplicatedLog().appendAndPersist(entry); + + if(entry.getData() instanceof ServerConfigurationPayload) { + context.updatePeerIds((ServerConfigurationPayload)entry.getData()); + } } LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); @@ -221,7 +232,7 @@ public class Follower extends AbstractRaftActorBehavior { } AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true, - lastIndex, lastTerm()); + lastIndex, lastTerm(), context.getPayloadVersion()); if(LOG.isTraceEnabled()) { LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply); @@ -231,13 +242,66 @@ public class Follower extends AbstractRaftActorBehavior { sender.tell(reply, actor()); - if (!context.isSnapshotCaptureInitiated()) { + 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 + + 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())) { + // 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)){ + 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 RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { return this; @@ -267,30 +331,38 @@ public class Follower extends AbstractRaftActorBehavior { if (message instanceof ElectionTimeout) { LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); - return switchBehavior(new Candidate(context)); + return internalSwitchBehavior(RaftState.Candidate); } else if (message instanceof InstallSnapshot) { InstallSnapshot installSnapshot = (InstallSnapshot) message; handleInstallSnapshot(sender, installSnapshot); } - scheduleElection(electionDuration()); + if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){ + scheduleElection(electionDuration()); + } return super.handleMessage(sender, message); } - private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) { - + private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot 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()); } + 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(), @@ -298,21 +370,33 @@ public class Follower extends AbstractRaftActorBehavior { installSnapshot.getLastIncludedIndex(), installSnapshot.getLastIncludedTerm(), installSnapshot.getLastIncludedIndex(), - installSnapshot.getLastIncludedTerm()); - - actor().tell(new ApplySnapshot(snapshot), actor()); + installSnapshot.getLastIncludedTerm(), + context.getTermInformation().getCurrentTerm(), + context.getTermInformation().getVotedFor(), + context.getPeerServerInfo()); - 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()); + 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); @@ -330,12 +414,13 @@ public class Follower extends AbstractRaftActorBehavior { } } - @Override public void close() throws Exception { + @Override + public void close() throws Exception { stopElection(); } @VisibleForTesting - ByteString getSnapshotChunksCollected(){ - return snapshotTracker != null ? snapshotTracker.getCollectedChunks() : ByteString.EMPTY; + SnapshotTracker getSnapshotTracker(){ + return snapshotTracker; } }