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%2FLeader.java;h=36e9b646e66f069504e8d220ceafb80bdc0a1b6a;hp=0dd39001136a25416c241eb8fb69085266d3752c;hb=f276ae33b951d173b51c467bb7bb1a5f5cf9a1e6;hpb=8cf40f4741c70a760dadb4300946c1dc88f95611 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java index 0dd3900113..36e9b646e6 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/Leader.java @@ -5,17 +5,22 @@ * terms of the Eclipse Public License v1.0 which accompanies this distribution, * and is available at http://www.eclipse.org/legal/epl-v10.html */ - package org.opendaylight.controller.cluster.raft.behaviors; import akka.actor.ActorRef; -import akka.actor.Cancellable; +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.base.messages.InitiateInstallSnapshot; -import org.opendaylight.controller.cluster.raft.base.messages.IsolatedLeaderCheck; -import scala.concurrent.duration.FiniteDuration; +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.messages.AppendEntriesReply; /** * The behavior of a RaftActor when it is in the Leader state @@ -40,77 +45,153 @@ import scala.concurrent.duration.FiniteDuration; * set commitIndex = N (§5.3, §5.4). */ public class Leader extends AbstractLeader { - private Cancellable installSnapshotSchedule = null; - private Cancellable isolatedLeaderCheckSchedule = null; - - public Leader(RaftActorContext context) { - super(context); + /** + * Internal message sent to periodically check if this leader has become isolated and should transition + * to {@link IsolatedLeader}. + */ + @VisibleForTesting + static final Object ISOLATED_LEADER_CHECK = new Object(); - scheduleInstallSnapshotCheck(context.getConfigParams().getIsolatedCheckInterval()); + private final Stopwatch isolatedLeaderCheck = Stopwatch.createStarted(); + private @Nullable LeadershipTransferContext leadershipTransferContext; - scheduleIsolatedLeaderCheck( - new FiniteDuration(context.getConfigParams().getHeartBeatInterval().length() * 10, - context.getConfigParams().getHeartBeatInterval().unit())); + public Leader(RaftActorContext context) { + super(context, RaftState.Leader); } - @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { + @Override + public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) { Preconditions.checkNotNull(sender, "sender should not be null"); - if (originalMessage instanceof IsolatedLeaderCheck) { + if (ISOLATED_LEADER_CHECK.equals(originalMessage)) { if (isLeaderIsolated()) { - LOG.info("At least {} followers need to be active, Switching {} from Leader to IsolatedLeader", - minIsolatedLeaderPeerCount, leaderId); - return switchBehavior(new IsolatedLeader(context)); + LOG.warn("{}: At least {} followers need to be active, Switching {} from Leader to IsolatedLeader", + context.getId(), getMinIsolatedLeaderPeerCount(), getLeaderId()); + return internalSwitchBehavior(RaftState.IsolatedLeader); + } else { + return this; } + } else { + return super.handleMessage(sender, originalMessage); } - - return super.handleMessage(sender, originalMessage); } - protected void stopInstallSnapshotSchedule() { - if (installSnapshotSchedule != null && !installSnapshotSchedule.isCancelled()) { - installSnapshotSchedule.cancel(); + @Override + protected void beforeSendHeartbeat(){ + if(isolatedLeaderCheck.elapsed(TimeUnit.MILLISECONDS) > context.getConfigParams().getIsolatedCheckIntervalInMillis()){ + context.getActor().tell(ISOLATED_LEADER_CHECK, context.getActor()); + isolatedLeaderCheck.reset().start(); + } + + if(leadershipTransferContext != null && leadershipTransferContext.isExpired( + context.getConfigParams().getElectionTimeOutInterval().toMillis())) { + LOG.debug("{}: Leadership transfer expired", logName()); + leadershipTransferContext = null; } } - protected void scheduleInstallSnapshotCheck(FiniteDuration interval) { - if(followers.size() == 0){ - // Optimization - do not bother scheduling a heartbeat as there are - // no followers + @Override + protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) { + RaftActorBehavior returnBehavior = super.handleAppendEntriesReply(sender, appendEntriesReply); + tryToCompleteLeadershipTransfer(appendEntriesReply.getFollowerId()); + return returnBehavior; + } + + /** + * Attempts to transfer leadership to a follower as per the raft paper (§3.10) as follows: + * + * + * @param leadershipTransferCohort + */ + public void transferLeadership(@Nonnull RaftActorLeadershipTransferCohort leadershipTransferCohort) { + 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; } - stopInstallSnapshotSchedule(); + long lastIndex = context.getReplicatedLog().lastIndex(); + boolean isVoting = context.getPeerInfo(followerId).isVoting(); - // Schedule a message to send append entries to followers that can - // accept an append entries with some data in it - installSnapshotSchedule = - context.getActorSystem().scheduler().scheduleOnce( - interval, - context.getActor(), new InitiateInstallSnapshot(), - context.getActorSystem().dispatcher(), context.getActor()); - } + LOG.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}, isVoting: {}", + logName(), followerId, followerInfo.getMatchIndex(), lastIndex, isVoting); + + if(isVoting && followerInfo.getMatchIndex() == lastIndex) { + LOG.debug("{}: Follower's log matches - sending ElectionTimeout", logName()); - protected void stopIsolatedLeaderCheckSchedule() { - if (isolatedLeaderCheckSchedule != null && !isolatedLeaderCheckSchedule.isCancelled()) { - isolatedLeaderCheckSchedule.cancel(); + // We can't be sure if the follower has applied all its log entries to its state so send an + // additional AppendEntries with the latest commit index. + sendAppendEntries(0, false); + + // Now send an ElectionTimeout to the matching follower to immediately start an election. + ActorSelection followerActor = context.getPeerActorSelection(followerId); + followerActor.tell(ElectionTimeout.INSTANCE, context.getActor()); + + LOG.debug("{}: Leader transfer complete", logName()); + + leadershipTransferContext.transferCohort.transferComplete(); + leadershipTransferContext = null; } } - protected void scheduleIsolatedLeaderCheck(FiniteDuration isolatedCheckInterval) { - isolatedLeaderCheckSchedule = context.getActorSystem().scheduler().schedule(isolatedCheckInterval, isolatedCheckInterval, - context.getActor(), new IsolatedLeaderCheck(), - context.getActorSystem().dispatcher(), context.getActor()); - } + @Override + public void close() { + if(leadershipTransferContext != null) { + leadershipTransferContext.transferCohort.abortTransfer(); + } - @Override public void close() throws Exception { - stopInstallSnapshotSchedule(); - stopIsolatedLeaderCheckSchedule(); super.close(); } @VisibleForTesting void markFollowerActive(String followerId) { - followerToLog.get(followerId).markFollowerActive(); + getFollower(followerId).markFollowerActive(); + } + + @VisibleForTesting + 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; + } } }