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=8aee8c1af88c31da507bede83065d3d457b76428;hp=6f107e9ae61ee060071c842810cb82c3e7f92f85;hb=d3c5dc3b0f6bea3fa1c2f964353b87d1a9fcaef8;hpb=913ae866cd0cc82991e1f66ac80f6a42b0daaa48 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 6f107e9ae6..8aee8c1af8 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 @@ -18,7 +18,9 @@ import akka.cluster.MemberStatus; import akka.japi.Procedure; 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; @@ -27,7 +29,6 @@ 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; -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.TimeoutNow; @@ -39,6 +40,7 @@ import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; +import org.opendaylight.controller.cluster.raft.persisted.Snapshot; /** * The behavior of a RaftActor in the Follower raft state. @@ -50,8 +52,6 @@ import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPay * */ public class Follower extends AbstractRaftActorBehavior { - private static final int SYNC_THRESHOLD = 10; - private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18; private final SyncStatusTracker initialSyncStatusTracker; @@ -61,16 +61,18 @@ public class Follower extends AbstractRaftActorBehavior { private String leaderId; private short leaderPayloadVersion; - public Follower(RaftActorContext context) { + public Follower(final RaftActorContext context) { this(context, null, (short)-1); } - public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) { + public Follower(final RaftActorContext context, final String initialLeaderId, + final short initialLeaderPayloadVersion) { super(context, RaftState.Follower); this.leaderId = initialLeaderId; this.leaderPayloadVersion = initialLeaderPayloadVersion; - initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); + initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams() + .getSyncIndexThreshold()); if (context.getPeerIds().isEmpty() && getLeaderId() == null) { actor().tell(TimeoutNow.INSTANCE, actor()); @@ -95,7 +97,7 @@ public class Follower extends AbstractRaftActorBehavior { } @VisibleForTesting - protected final void setLeaderPayloadVersion(short leaderPayloadVersion) { + protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) { this.leaderPayloadVersion = leaderPayloadVersion; } @@ -107,7 +109,7 @@ public class Follower extends AbstractRaftActorBehavior { lastLeaderMessageTimer.start(); } - private boolean isLogEntryPresent(long index) { + private boolean isLogEntryPresent(final long index) { if (context.getReplicatedLog().isInSnapshot(index)) { return true; } @@ -117,12 +119,12 @@ public class Follower extends AbstractRaftActorBehavior { } - private void updateInitialSyncStatus(long currentLeaderCommit, String newLeaderId) { + private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) { initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex()); } @Override - protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) { + protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) { int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0; if (log.isTraceEnabled()) { @@ -138,7 +140,7 @@ public class Follower extends AbstractRaftActorBehavior { 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()); - snapshotTracker = null; + closeSnapshotTracker(); } if (snapshotTracker != null || context.getSnapshotManager().isApplying()) { @@ -156,7 +158,6 @@ 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 long lastIndex = lastIndex(); @@ -164,11 +165,12 @@ public class Follower extends AbstractRaftActorBehavior { // We found that the log was out of sync so just send a negative // reply and return - log.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}", - logName(), lastIndex, lastTerm()); + final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, + lastTerm(), context.getPayloadVersion()); - sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, - lastTerm(), context.getPayloadVersion()), actor()); + log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + sender.tell(reply, actor()); return this; } @@ -205,8 +207,7 @@ public class Follower extends AbstractRaftActorBehavior { if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) { - log.debug("{}: Removing entries from log starting at {}", logName(), - matchEntry.getIndex()); + log.info("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex()); // Entries do not match so remove all subsequent entries if (!context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) { @@ -215,7 +216,8 @@ public class Follower extends AbstractRaftActorBehavior { // 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()); + log.info("{}: Could not remove entries - sending reply to force snapshot", logName()); + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, lastTerm(), context.getPayloadVersion(), true), actor()); return this; @@ -223,6 +225,7 @@ public class Follower extends AbstractRaftActorBehavior { break; } else { + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex, lastTerm(), context.getPayloadVersion(), true), actor()); return this; @@ -283,13 +286,26 @@ public class Follower extends AbstractRaftActorBehavior { log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex()); } + 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); + } + + // Reply to the leader before applying any previous state so as not to hold up leader consensus. + updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId()); + sender.tell(reply, actor()); + // 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(): {}," + log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, " + "context.getLastApplied(): {}, lastIndex(): {}", logName(), appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex); } @@ -297,17 +313,6 @@ public class Follower extends AbstractRaftActorBehavior { applyLogToStateMachine(appendEntries.getLeaderCommit()); } - 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()); } @@ -315,73 +320,80 @@ public class Follower extends AbstractRaftActorBehavior { 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; + private boolean isOutOfSync(final AppendEntries appendEntries) { + final 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 - " - + "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.debug("{}: 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.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; + // 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()); + return true; + } + + if (lastIndex > -1) { + if (isLogEntryPresent(appendEntries.getPrevLogIndex())) { + final long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex()); + if (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()); + 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()); + return true; + } } - return outOfSync; + + 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 + + log.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the " + + "in-memory journal", logName(), appendEntries.getReplicatedToAllIndex()); + return true; + } + + final List entries = appendEntries.getEntries(); + if (entries != null && 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); + return true; + } + } + + return false; } @Override - protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, - AppendEntriesReply appendEntriesReply) { + protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender, + final AppendEntriesReply appendEntriesReply) { return this; } @Override - protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, + final RequestVoteReply requestVoteReply) { return this; } @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object message) { + public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { if (message instanceof ElectionTimeout || message instanceof TimeoutNow) { return handleElectionTimeout(message); } @@ -396,7 +408,7 @@ public class Follower extends AbstractRaftActorBehavior { // 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", + log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term", logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); context.getTermInformation().updateAndPersist(rpc.getTerm(), null); @@ -417,7 +429,7 @@ public class Follower extends AbstractRaftActorBehavior { return super.handleMessage(sender, rpc); } - private RaftActorBehavior handleElectionTimeout(Object message) { + private RaftActorBehavior handleElectionTimeout(final Object message) { // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader // during the election timeout interval. It may that the election timer expired b/c this actor // was busy and messages got delayed, in which case leader messages would be backed up in the @@ -510,14 +522,15 @@ public class Follower extends AbstractRaftActorBehavior { return false; } - private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { + private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) { log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); leaderId = installSnapshot.getLeaderId(); if (snapshotTracker == null) { - snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId()); + snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(), + context); } updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); @@ -528,7 +541,11 @@ public class Follower extends AbstractRaftActorBehavior { if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(), installSnapshot.getLastChunkHashCode())) { - Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(), + + log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId()); + + Snapshot snapshot = Snapshot.create( + context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()), new ArrayList<>(), installSnapshot.getLastIncludedIndex(), installSnapshot.getLastIncludedTerm(), @@ -554,24 +571,32 @@ public class Follower extends AbstractRaftActorBehavior { actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor()); - snapshotTracker = null; + closeSnapshotTracker(); } else { log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply); sender.tell(reply, actor()); } - } catch (SnapshotTracker.InvalidChunkException e) { + } catch (IOException e) { log.debug("{}: Exception in InstallSnapshot of follower", logName(), e); sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor()); - snapshotTracker = null; + closeSnapshotTracker(); + } + } + + private void closeSnapshotTracker() { + if (snapshotTracker != null) { + snapshotTracker.close(); + snapshotTracker = null; } } @Override public void close() { + closeSnapshotTracker(); stopElection(); }