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=7e6654d5d6b9849b106a36bf56c76928798d912d;hp=974ec475856af94f6e36ad367b7cf555d71354bd;hb=f276ae33b951d173b51c467bb7bb1a5f5cf9a1e6;hpb=a93bcbe711f66ef6ec7bc97972f108859c87a11e 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 974ec47585..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,17 @@ 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.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.base.messages.FollowerInitialSyncUpStatus; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; @@ -37,24 +39,59 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; * */ public class Follower extends AbstractRaftActorBehavior { + private static final int SYNC_THRESHOLD = 10; - private SnapshotTracker snapshotTracker = null; + private final SyncStatusTracker initialSyncStatusTracker; + + private final Procedure appendAndPersistCallback = new Procedure() { + @Override + public void apply(ReplicatedLogEntry logEntry) { + context.getReplicatedLog().captureSnapshotIfReady(logEntry); + } + }; - private final InitialSyncStatusTracker initialSyncStatusTracker; + private SnapshotTracker snapshotTracker = null; + private String leaderId; + private short leaderPayloadVersion; public Follower(RaftActorContext 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 InitialSyncStatusTracker(context.getActor()); + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); - if(context.getRaftPolicy().automaticElectionsEnabled()) { - if (context.getPeerAddresses().isEmpty()) { - actor().tell(ELECTION_TIMEOUT, actor()); + if(canStartElection()) { + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(ElectionTimeout.INSTANCE, actor()); } else { scheduleElection(electionDuration()); } } + } + @Override + public final String getLeaderId() { + return leaderId; + } + + @VisibleForTesting + protected final void setLeaderId(final String leaderId) { + this.leaderId = Preconditions.checkNotNull(leaderId); + } + + @Override + public short getLeaderPayloadVersion() { + return leaderPayloadVersion; + } + + @VisibleForTesting + protected final void setLeaderPayloadVersion(short leaderPayloadVersion) { + this.leaderPayloadVersion = leaderPayloadVersion; } private boolean isLogEntryPresent(long index){ @@ -88,8 +125,8 @@ public class Follower extends AbstractRaftActorBehavior { 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()) { @@ -102,7 +139,7 @@ public class Follower extends AbstractRaftActorBehavior { // to make it easier to read. Before refactoring ensure tests // cover the code properly - if (snapshotTracker != null) { + 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()); @@ -117,8 +154,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 @@ -187,7 +223,11 @@ 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()); + } } LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size()); @@ -272,7 +312,8 @@ public class Follower extends AbstractRaftActorBehavior { logName(), prevLogTerm, appendEntries.getPrevLogTerm()); } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1 && appendEntries.getReplicatedToAllIndex() != -1 - && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())) { + && !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 @@ -280,8 +321,9 @@ public class Follower extends AbstractRaftActorBehavior { "{}: 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)){ + && 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); @@ -291,54 +333,65 @@ 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) { + @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; + } + } - Object message = fromSerializableMessage(originalMessage); + final Object message = fromSerializableMessage(originalMessage); + if (!(message instanceof RaftRPC)) { + // The rest of the processing requires the message to be a RaftRPC + return null; + } - 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()); + 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); - } + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); } - if (message instanceof ElectionTimeout) { - LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); - return internalSwitchBehavior(RaftState.Candidate); - - } 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(ActorRef sender, InstallSnapshot installSnapshot) { + private void handleInstallSnapshot(final ActorRef sender, 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()); @@ -347,6 +400,9 @@ public class Follower extends AbstractRaftActorBehavior { 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(), @@ -354,21 +410,33 @@ public class Follower extends AbstractRaftActorBehavior { installSnapshot.getLastIncludedIndex(), installSnapshot.getLastIncludedTerm(), installSnapshot.getLastIncludedIndex(), - installSnapshot.getLastIncludedTerm()); + installSnapshot.getLastIncludedTerm(), + context.getTermInformation().getCurrentTerm(), + context.getTermInformation().getVotedFor(), + context.getPeerServerInfo(true)); - actor().tell(new ApplySnapshot(snapshot), actor()); + ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() { + @Override + public void onSuccess() { + LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); - snapshotTracker = null; - - } + 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); @@ -387,7 +455,7 @@ public class Follower extends AbstractRaftActorBehavior { } @Override - public void close() throws Exception { + public void close() { stopElection(); } @@ -395,36 +463,4 @@ public class Follower extends AbstractRaftActorBehavior { 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; - } - } - } - } }