package org.opendaylight.controller.cluster.raft.behaviors;
import akka.actor.ActorRef;
+import akka.actor.ActorSelection;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Stopwatch;
import java.util.concurrent.TimeUnit;
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
import org.opendaylight.controller.cluster.raft.RaftActorContext;
+import org.opendaylight.controller.cluster.raft.RaftActorLeadershipTransferCohort;
import org.opendaylight.controller.cluster.raft.RaftState;
+import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
import org.opendaylight.controller.cluster.raft.base.messages.IsolatedLeaderCheck;
+import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
/**
* The behavior of a RaftActor when it is in the Leader state
public class Leader extends AbstractLeader {
private static final IsolatedLeaderCheck ISOLATED_LEADER_CHECK = new IsolatedLeaderCheck();
private final Stopwatch isolatedLeaderCheck;
+ private @Nullable LeadershipTransferContext leadershipTransferContext;
public Leader(RaftActorContext context) {
super(context);
isolatedLeaderCheck.reset().start();
}
+ if(leadershipTransferContext != null && leadershipTransferContext.isExpired(
+ context.getConfigParams().getElectionTimeOutInterval().toMillis())) {
+ LOG.debug("{}: Leadership transfer expired", logName());
+ leadershipTransferContext = null;
+ }
+ }
+
+ @Override
+ protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
+ RaftActorBehavior returnBehavior = super.handleAppendEntriesReply(sender, appendEntriesReply);
+ tryToCompleteLeadershipTransfer(appendEntriesReply.getFollowerId());
+ return returnBehavior;
+ }
+
+ public void transferLeadership(@Nonnull RaftActorLeadershipTransferCohort leadershipTransferCohort) {
+ if(!context.hasFollowers()) {
+ leadershipTransferCohort.transferComplete();
+ return;
+ }
+
+ LOG.debug("{}: Attempting to transfer leadership", logName());
+
+ leadershipTransferContext = new LeadershipTransferContext(leadershipTransferCohort);
+
+ // Send an immediate heart beat to the followers.
+ sendAppendEntries(0, false);
+ }
+
+ private void tryToCompleteLeadershipTransfer(String followerId) {
+ if(leadershipTransferContext == null) {
+ return;
+ }
+
+ FollowerLogInformation followerInfo = getFollower(followerId);
+ if(followerInfo == null) {
+ return;
+ }
+
+ long lastIndex = context.getReplicatedLog().lastIndex();
+
+ LOG.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}",
+ logName(), followerId, followerInfo.getMatchIndex(), lastIndex);
+
+ if(followerInfo.getMatchIndex() == lastIndex) {
+ LOG.debug("{}: Follower's log matches - sending ElectionTimeout", logName());
+
+ // We can't be sure if the follower has applied all its log entries to its state so send an
+ // additional AppendEntries.
+ sendAppendEntries(0, false);
+
+ // Now send an ElectionTimeout to the matching follower to immediately start an election.
+ ActorSelection followerActor = context.getPeerActorSelection(followerId);
+ followerActor.tell(new ElectionTimeout(), context.getActor());
+
+ LOG.debug("{}: Leader transfer complete", logName());
+
+ leadershipTransferContext.transferCohort.transferComplete();
+ leadershipTransferContext = null;
+ }
}
@Override
public void close() throws Exception {
+ if(leadershipTransferContext != null) {
+ leadershipTransferContext.transferCohort.abortTransfer();
+ }
+
super.close();
}
void markFollowerInActive(String followerId) {
getFollower(followerId).markFollowerInActive();
}
+
+ private static class LeadershipTransferContext {
+ RaftActorLeadershipTransferCohort transferCohort;
+ Stopwatch timer = Stopwatch.createStarted();
+
+ LeadershipTransferContext(RaftActorLeadershipTransferCohort transferCohort) {
+ this.transferCohort = transferCohort;
+ }
+
+ boolean isExpired(long timeout) {
+ if(timer.elapsed(TimeUnit.MILLISECONDS) >= timeout) {
+ transferCohort.abortTransfer();
+ return true;
+ }
+
+ return false;
+ }
+ }
}