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=1a721335068336c7b9d00e95da720d2fba48ee80;hp=a9d1b8233e5aa2301998375652fea36fe48442ee;hb=95d7b8820236d16cb7e37c4a95fcae6f6d55581e;hpb=23b10ec4ddfdd9348c2abe7dbcfbed3b49db3dc6 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 a9d1b8233e..1a72133506 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,17 +9,27 @@ package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; +import akka.actor.ActorSelection; +import akka.actor.Address; +import akka.cluster.Cluster; +import akka.cluster.ClusterEvent.CurrentClusterState; +import akka.cluster.Member; +import akka.cluster.MemberStatus; import akka.japi.Procedure; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; +import com.google.common.base.Stopwatch; import java.util.ArrayList; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +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.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.TimeoutNow; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; @@ -27,6 +37,7 @@ 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; +import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload; /** * The behavior of a RaftActor in the Follower state @@ -41,15 +52,14 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; 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; - private final Procedure appendAndPersistCallback = new Procedure() { - @Override - public void apply(ReplicatedLogEntry logEntry) { - context.getReplicatedLog().captureSnapshotIfReady(logEntry); - } - }; + private final Procedure appendAndPersistCallback = + logEntry -> context.getReplicatedLog().captureSnapshotIfReady(logEntry); + private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted(); private SnapshotTracker snapshotTracker = null; private String leaderId; private short leaderPayloadVersion; @@ -65,12 +75,10 @@ public class Follower extends AbstractRaftActorBehavior { initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD); - if(canStartElection()) { - if (context.getPeerIds().isEmpty() && getLeaderId() == null) { - actor().tell(ElectionTimeout.INSTANCE, actor()); - } else { - scheduleElection(electionDuration()); - } + if (context.getPeerIds().isEmpty() && getLeaderId() == null) { + actor().tell(TimeoutNow.INSTANCE, actor()); + } else { + scheduleElection(electionDuration()); } } @@ -80,8 +88,8 @@ public class Follower extends AbstractRaftActorBehavior { } @VisibleForTesting - protected final void setLeaderId(final String leaderId) { - this.leaderId = Preconditions.checkNotNull(leaderId); + protected final void setLeaderId(@Nullable final String leaderId) { + this.leaderId = leaderId; } @Override @@ -94,31 +102,22 @@ public class Follower extends AbstractRaftActorBehavior { this.leaderPayloadVersion = leaderPayloadVersion; } - private boolean isLogEntryPresent(long index){ - if(context.getReplicatedLog().isInSnapshot(index)) { - return true; + private void restartLastLeaderMessageTimer() { + if (lastLeaderMessageTimer.isRunning()) { + lastLeaderMessageTimer.reset(); } - ReplicatedLogEntry previousEntry = context.getReplicatedLog() - .get(index); - - return previousEntry != null; - + lastLeaderMessageTimer.start(); } - 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){ @@ -139,6 +138,12 @@ public class Follower extends AbstractRaftActorBehavior { // to make it easier to read. Before refactoring ensure tests // cover the code properly + 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; + } + 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, @@ -186,14 +191,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; } @@ -203,7 +214,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, @@ -214,8 +236,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++) { @@ -344,17 +366,11 @@ public class Follower extends AbstractRaftActorBehavior { } @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; - } + public RaftActorBehavior handleMessage(ActorRef sender, Object message) { + if (message instanceof ElectionTimeout || message instanceof TimeoutNow) { + return handleElectionTimeout(message); } - final Object message = fromSerializableMessage(originalMessage); if (!(message instanceof RaftRPC)) { // The rest of the processing requires the message to be a RaftRPC return null; @@ -372,19 +388,113 @@ public class Follower extends AbstractRaftActorBehavior { } if (rpc instanceof InstallSnapshot) { - InstallSnapshot installSnapshot = (InstallSnapshot) rpc; - handleInstallSnapshot(sender, installSnapshot); + handleInstallSnapshot(sender, (InstallSnapshot) rpc); + restartLastLeaderMessageTimer(); scheduleElection(electionDuration()); return this; } if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) { + restartLastLeaderMessageTimer(); scheduleElection(electionDuration()); } return super.handleMessage(sender, rpc); } + private RaftActorBehavior handleElectionTimeout(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 + // queue but would be processed before the ElectionTimeout message and thus would restart the + // lastLeaderMessageTimer. + long lastLeaderMessageInterval = lastLeaderMessageTimer.elapsed(TimeUnit.MILLISECONDS); + long electionTimeoutInMillis = context.getConfigParams().getElectionTimeOutInterval().toMillis(); + boolean noLeaderMessageReceived = !lastLeaderMessageTimer.isRunning() || + lastLeaderMessageInterval >= electionTimeoutInMillis; + + if(canStartElection()) { + if(message instanceof TimeoutNow) { + LOG.debug("{}: Received TimeoutNow - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } else if(noLeaderMessageReceived) { + // Check the cluster state to see if the leader is known to be up before we go to Candidate. + // However if we haven't heard from the leader in a long time even though the cluster state + // indicates it's up then something is wrong - leader might be stuck indefinitely - so switch + // to Candidate, + long maxElectionTimeout = electionTimeoutInMillis * MAX_ELECTION_TIMEOUT_FACTOR; + if(isLeaderAvailabilityKnown() && lastLeaderMessageInterval < maxElectionTimeout) { + LOG.debug("{}: Received ElectionTimeout but leader appears to be available", logName()); + scheduleElection(electionDuration()); + } else { + LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName()); + return internalSwitchBehavior(RaftState.Candidate); + } + } else { + LOG.debug("{}: Received ElectionTimeout but lastLeaderMessageInterval {} < election timeout {}", + logName(), lastLeaderMessageInterval, context.getConfigParams().getElectionTimeOutInterval()); + scheduleElection(electionDuration()); + } + } else if(message instanceof ElectionTimeout) { + if(noLeaderMessageReceived) { + setLeaderId(null); + } + + scheduleElection(electionDuration()); + } + + return this; + } + + private boolean isLeaderAvailabilityKnown() { + if(leaderId == null) { + return false; + } + + Optional cluster = context.getCluster(); + if(!cluster.isPresent()) { + return false; + } + + ActorSelection leaderActor = context.getPeerActorSelection(leaderId); + if(leaderActor == null) { + return false; + } + + Address leaderAddress = leaderActor.anchorPath().address(); + + CurrentClusterState state = cluster.get().state(); + Set unreachable = state.getUnreachable(); + + LOG.debug("{}: Checking for leader {} in the cluster unreachable set {}", logName(), leaderAddress, + unreachable); + + for(Member m: unreachable) { + if(leaderAddress.equals(m.address())) { + LOG.info("{}: Leader {} is unreachable", logName(), leaderAddress); + return false; + } + } + + for(Member m: state.getMembers()) { + if(leaderAddress.equals(m.address())) { + if(m.status() == MemberStatus.up() || m.status() == MemberStatus.weaklyUp()) { + LOG.debug("{}: Leader {} cluster status is {} - leader is available", logName(), + leaderAddress, m.status()); + return true; + } else { + LOG.debug("{}: Leader {} cluster status is {} - leader is unavailable", logName(), + leaderAddress, m.status()); + return false; + } + } + } + + LOG.debug("{}: Leader {} not found in the cluster member set", logName(), leaderAddress); + + return false; + } + private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) { LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot); @@ -392,7 +502,7 @@ public class Follower extends AbstractRaftActorBehavior { leaderId = installSnapshot.getLeaderId(); if(snapshotTracker == null){ - snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks()); + snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId()); } updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId()); @@ -411,7 +521,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