Measure follower activity in nanoseconds
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Leader.java
index 21aad966cb2b7dd375b6e4667e952ab4b85891a9..eb49abc17aea4a0ebefacde0a5925312082e1405 100644 (file)
@@ -10,6 +10,7 @@ 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.Optional;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 import java.util.concurrent.TimeUnit;
@@ -19,13 +20,13 @@ 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.base.messages.TimeoutNow;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 
 /**
- * The behavior of a RaftActor when it is in the Leader state
- * <p/>
+ * The behavior of a RaftActor when it is in the Leader state.
+ *
+ * <p>
  * Leaders:
  * <ul>
  * <li> Upon election: send initial empty AppendEntries RPCs
@@ -35,51 +36,62 @@ import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
  * respond after entry applied to state machine (§5.3)
  * <li> If last log index ≥ nextIndex for a follower: send
  * AppendEntries RPC with log entries starting at nextIndex
- * <ul>
  * <li> If successful: update nextIndex and matchIndex for
  * follower (§5.3)
  * <li> If AppendEntries fails because of log inconsistency:
  * decrement nextIndex and retry (§5.3)
- * </ul>
- * <li> If there exists an N such that N > commitIndex, a majority
+ * <li> If there exists an N such that N &gt; commitIndex, a majority
  * of matchIndex[i] ≥ N, and log[N].term == currentTerm:
  * set commitIndex = N (§5.3, §5.4).
+ * </ul>
  */
 public class Leader extends AbstractLeader {
-    private static final IsolatedLeaderCheck ISOLATED_LEADER_CHECK = new IsolatedLeaderCheck();
-    private final Stopwatch isolatedLeaderCheck;
-    private @Nullable LeadershipTransferContext leadershipTransferContext;
+    /**
+     * 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();
+
+    private final Stopwatch isolatedLeaderCheck = Stopwatch.createStarted();
+    @Nullable private LeadershipTransferContext leadershipTransferContext;
+
+    Leader(RaftActorContext context, @Nullable AbstractLeader initializeFromLeader) {
+        super(context, RaftState.Leader, initializeFromLeader);
+    }
 
     public Leader(RaftActorContext context) {
-        super(context);
-        isolatedLeaderCheck = Stopwatch.createStarted();
+        this(context, null);
     }
 
-    @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.warn("{}: At least {} followers need to be active, Switching {} from Leader to IsolatedLeader",
-                        context.getId(), getMinIsolatedLeaderPeerCount(), leaderId);
-
-                return internalSwitchBehavior(RaftState.IsolatedLeader);
+                log.warn("{}: At least {} followers need to be active, Switching {} from Leader to IsolatedLeader",
+                    context.getId(), getMinIsolatedLeaderPeerCount(), getLeaderId());
+                return internalSwitchBehavior(new IsolatedLeader(context, this));
+            } else {
+                return this;
             }
+        } else {
+            return super.handleMessage(sender, originalMessage);
         }
-
-        return super.handleMessage(sender, originalMessage);
     }
 
     @Override
-    protected void beforeSendHeartbeat(){
-        if(isolatedLeaderCheck.elapsed(TimeUnit.MILLISECONDS) > context.getConfigParams().getIsolatedCheckIntervalInMillis()){
+    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(
+        if (leadershipTransferContext != null && leadershipTransferContext.isExpired(
                 context.getConfigParams().getElectionTimeOutInterval().toMillis())) {
-            LOG.debug("{}: Leadership transfer expired", logName());
+            log.debug("{}: Leadership transfer expired", logName());
             leadershipTransferContext = null;
         }
     }
@@ -97,20 +109,20 @@ public class Leader extends AbstractLeader {
      * <li>Start a timer (Stopwatch).</li>
      * <li>Send an initial AppendEntries heartbeat to all followers.</li>
      * <li>On AppendEntriesReply, check if the follower's new match Index matches the leader's last index</li>
-     * <li>If it matches, </li>
+     * <li>If it matches,
      *   <ul>
      *   <li>Send an additional AppendEntries to ensure the follower has applied all its log entries to its state.</li>
      *   <li>Send an ElectionTimeout to the follower to immediately start an election.</li>
      *   <li>Notify {@link RaftActorLeadershipTransferCohort#transferComplete}.</li>
-     *   </ul>
+     *   </ul></li>
      * <li>Otherwise if the election time out period elapses, notify
-     *     {@link RaftActorLeadershipTransferCohort#abortTtransfer}.</li>
+     *     {@link RaftActorLeadershipTransferCohort#abortTransfer}.</li>
      * </ul>
      *
-     * @param leadershipTransferCohort
+     * @param leadershipTransferCohort the cohort participating in the leadership transfer
      */
     public void transferLeadership(@Nonnull RaftActorLeadershipTransferCohort leadershipTransferCohort) {
-        LOG.debug("{}: Attempting to transfer leadership", logName());
+        log.debug("{}: Attempting to transfer leadership", logName());
 
         leadershipTransferContext = new LeadershipTransferContext(leadershipTransferCohort);
 
@@ -119,32 +131,40 @@ public class Leader extends AbstractLeader {
     }
 
     private void tryToCompleteLeadershipTransfer(String followerId) {
-        if(leadershipTransferContext == null) {
+        if (leadershipTransferContext == null) {
+            return;
+        }
+
+        final Optional<String> requestedFollowerIdOptional
+                = leadershipTransferContext.transferCohort.getRequestedFollowerId();
+        if (requestedFollowerIdOptional.isPresent() && !requestedFollowerIdOptional.get().equals(followerId)) {
+            // we want to transfer leadership to specific follower
             return;
         }
 
         FollowerLogInformation followerInfo = getFollower(followerId);
-        if(followerInfo == null) {
+        if (followerInfo == null) {
             return;
         }
 
         long lastIndex = context.getReplicatedLog().lastIndex();
+        boolean isVoting = context.getPeerInfo(followerId).isVoting();
 
-        LOG.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}",
-                logName(), followerId, followerInfo.getMatchIndex(), lastIndex);
+        log.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}, isVoting: {}",
+                logName(), followerId, followerInfo.getMatchIndex(), lastIndex, isVoting);
 
-        if(followerInfo.getMatchIndex() == lastIndex) {
-            LOG.debug("{}: Follower's log matches - sending ElectionTimeout", logName());
+        if (isVoting && 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 with the latest commit index.
             sendAppendEntries(0, false);
 
-            // Now send an ElectionTimeout to the matching follower to immediately start an election.
+            // Now send a TimeoutNow message to the matching follower to immediately start an election.
             ActorSelection followerActor = context.getPeerActorSelection(followerId);
-            followerActor.tell(new ElectionTimeout(), context.getActor());
+            followerActor.tell(TimeoutNow.INSTANCE, context.getActor());
 
-            LOG.debug("{}: Leader transfer complete", logName());
+            log.debug("{}: Leader transfer complete", logName());
 
             leadershipTransferContext.transferCohort.transferComplete();
             leadershipTransferContext = null;
@@ -152,9 +172,11 @@ public class Leader extends AbstractLeader {
     }
 
     @Override
-    public void close() throws Exception {
-        if(leadershipTransferContext != null) {
-            leadershipTransferContext.transferCohort.abortTransfer();
+    public void close() {
+        if (leadershipTransferContext != null) {
+            LeadershipTransferContext localLeadershipTransferContext = leadershipTransferContext;
+            leadershipTransferContext = null;
+            localLeadershipTransferContext.transferCohort.abortTransfer();
         }
 
         super.close();
@@ -179,7 +201,7 @@ public class Leader extends AbstractLeader {
         }
 
         boolean isExpired(long timeout) {
-            if(timer.elapsed(TimeUnit.MILLISECONDS) >= timeout) {
+            if (timer.elapsed(TimeUnit.MILLISECONDS) >= timeout) {
                 transferCohort.abortTransfer();
                 return true;
             }