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=e4d42661de3a9612d558e29685c55bf195d9a331;hb=refs%2Fchanges%2F35%2F41535%2F2;hp=bcc2480f459318f35ce3aa30e2d818404626d99b;hpb=252ba03242407ee584c38fafdbfa1c322e66151d;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 bcc2480f45..e4d42661de 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,8 +9,10 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.japi.Procedure; import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; +import javax.annotation.Nullable; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry; @@ -37,66 +39,75 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; * */ public class Follower extends AbstractRaftActorBehavior { - - private SnapshotTracker snapshotTracker = null; + private static final int SYNC_THRESHOLD = 10; private final SyncStatusTracker initialSyncStatusTracker; - private static final int SYNC_THRESHOLD = 10; + 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) { - this(context, null); + this(context, null, (short)-1); } - public Follower(RaftActorContext context, String initialLeaderId) { + public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) { super(context, RaftState.Follower); - leaderId = initialLeaderId; + this.leaderId = initialLeaderId; + this.leaderPayloadVersion = initialLeaderPayloadVersion; initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); - if(canStartElection()) { - if (context.getPeerIds().isEmpty() && getLeaderId() == null) { - actor().tell(ELECTION_TIMEOUT, actor()); - } else { - scheduleElection(electionDuration()); - } + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(ElectionTimeout.INSTANCE, actor()); + } else { + scheduleElection(electionDuration()); } - } - private boolean isLogEntryPresent(long index){ - if(index == context.getReplicatedLog().getSnapshotIndex()){ - return true; - } + @Override + public final String getLeaderId() { + return leaderId; + } - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(index); + @VisibleForTesting + protected final void setLeaderId(@Nullable final String leaderId) { + this.leaderId = leaderId; + } - return previousEntry != null; + @Override + public short getLeaderPayloadVersion() { + return leaderPayloadVersion; + } + @VisibleForTesting + protected final void setLeaderPayloadVersion(short leaderPayloadVersion) { + this.leaderPayloadVersion = leaderPayloadVersion; } - private long getLogEntryTerm(long index){ - if(index == context.getReplicatedLog().getSnapshotIndex()){ - return context.getReplicatedLog().getSnapshotTerm(); + private boolean isLogEntryPresent(long index){ + if(context.getReplicatedLog().isInSnapshot(index)) { + return true; } - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(index); + ReplicatedLogEntry entry = context.getReplicatedLog().get(index); + return entry != null; - if(previousEntry != null){ - return previousEntry.getTerm(); - } - - return -1; } private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){ initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex()); } - @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender, - AppendEntries appendEntries) { + @Override + protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; if(LOG.isTraceEnabled()) { @@ -124,8 +135,7 @@ public class Follower extends AbstractRaftActorBehavior { // If we got here then we do appear to be talking to the leader leaderId = appendEntries.getLeaderId(); - - setLeaderPayloadVersion(appendEntries.getPayloadVersion()); + leaderPayloadVersion = appendEntries.getPayloadVersion(); updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); // First check if the logs are in sync or not @@ -157,14 +167,20 @@ public class Follower extends AbstractRaftActorBehavior { // 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 + if(!isLogEntryPresent(matchEntry.getIndex())) { + // newEntry not found in the log break; } - if (newEntry.getTerm() == matchEntry.getTerm()) { + 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; } @@ -174,7 +190,18 @@ public class Follower extends AbstractRaftActorBehavior { matchEntry.getIndex()); // Entries do not match so remove all subsequent entries - context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex()); + 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. + + LOG.debug("{}: 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; + } + break; } else { sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, @@ -185,8 +212,8 @@ public class Follower extends AbstractRaftActorBehavior { } lastIndex = lastIndex(); - LOG.debug("{}: After cleanup entries to be added from = {}", logName(), - (addEntriesFrom + lastIndex)); + LOG.debug("{}: After cleanup, lastIndex: {}, entries to be added from: {}", logName(), + lastIndex, addEntriesFrom); // 4. Append any new entries not already in the log for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) { @@ -194,7 +221,7 @@ public class Follower extends AbstractRaftActorBehavior { LOG.debug("{}: Append entry to log {}", logName(), entry.getData()); - context.getReplicatedLog().appendAndPersist(entry); + context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback); if(entry.getData() instanceof ServerConfigurationPayload) { context.updatePeerIds((ServerConfigurationPayload)entry.getData()); @@ -270,8 +297,8 @@ public class Follower extends AbstractRaftActorBehavior { // 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()); + LOG.debug("{}: 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 @@ -283,8 +310,7 @@ public class Follower extends AbstractRaftActorBehavior { logName(), prevLogTerm, appendEntries.getPrevLogTerm()); } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 && appendEntries.getReplicatedToAllIndex() != -1 - && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex()) - && !context.getReplicatedLog().isInSnapshot(appendEntries.getReplicatedToAllIndex())) { + && !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 @@ -293,8 +319,7 @@ public class Follower extends AbstractRaftActorBehavior { 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)) { + && !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); @@ -304,51 +329,60 @@ public class Follower extends AbstractRaftActorBehavior { return outOfSync; } - @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, + @Override + protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { return this; } - @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, + @Override + protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, RequestVoteReply requestVoteReply) { return this; } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { - - Object message = fromSerializableMessage(originalMessage); - - 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()); - - context.getTermInformation().updateAndPersist(rpc.getTerm(), null); - } - } - - if (message instanceof ElectionTimeout) { - if(canStartElection()) { + @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 { + setLeaderId(null); + scheduleElection(electionDuration()); return this; } + } + + final Object message = fromSerializableMessage(originalMessage); + 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()) { + LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", + logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); + + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); + } - } else if (message instanceof InstallSnapshot) { - InstallSnapshot installSnapshot = (InstallSnapshot) message; + if (rpc instanceof InstallSnapshot) { + InstallSnapshot installSnapshot = (InstallSnapshot) rpc; handleInstallSnapshot(sender, installSnapshot); + scheduleElection(electionDuration()); + return this; } - if(message instanceof RaftRPC && (!(message instanceof RequestVote) || (canGrantVote((RequestVote) message)))){ + if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) { scheduleElection(electionDuration()); } - return super.handleMessage(sender, message); + return super.handleMessage(sender, rpc); } private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { @@ -377,7 +411,7 @@ public class Follower extends AbstractRaftActorBehavior { installSnapshot.getLastIncludedTerm(), context.getTermInformation().getCurrentTerm(), context.getTermInformation().getVotedFor(), - context.getPeerServerInfo(true)); + installSnapshot.getServerConfig().orNull()); ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() { @Override @@ -419,7 +453,7 @@ public class Follower extends AbstractRaftActorBehavior { } @Override - public void close() throws Exception { + public void close() { stopElection(); }