Further Guava Optional cleanups
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Leader.java
index a50666233c31f30b2e94cbf4c49d53a95cca93f4..3534ac5cf142eda058ceb1e39b90d311b096b20b 100644 (file)
@@ -5,44 +5,29 @@
  * 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 static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
-import akka.actor.Cancellable;
-import com.google.common.base.Preconditions;
-import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
-import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import org.eclipse.jdt.annotation.NonNull;
+import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
-import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl;
 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.ReplicatedLogEntry;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
-import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
-import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
-import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
+import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
-import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
-import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
-import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
-import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
-import scala.concurrent.duration.FiniteDuration;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * 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
@@ -52,376 +37,178 @@ import java.util.concurrent.atomic.AtomicLong;
  * 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 AbstractRaftActorBehavior {
-
-
-    private final Map<String, FollowerLogInformation> followerToLog =
-        new HashMap();
-
-    private final Set<String> followers;
-
-    private Cancellable heartbeatSchedule = null;
-    private Cancellable appendEntriesSchedule = null;
-    private Cancellable installSnapshotSchedule = null;
-
-    private List<ClientRequestTracker> trackerList = new ArrayList<>();
-
-    private final int minReplicationCount;
-
-    public Leader(RaftActorContext context) {
-        super(context);
-
-        if (lastIndex() >= 0) {
-            context.setCommitIndex(lastIndex());
-        }
-
-        followers = context.getPeerAddresses().keySet();
-
-        for (String followerId : followers) {
-            FollowerLogInformation followerLogInformation =
-                new FollowerLogInformationImpl(followerId,
-                    new AtomicLong(lastIndex()),
-                    new AtomicLong(-1));
-
-            followerToLog.put(followerId, followerLogInformation);
-        }
-
-        context.getLogger().debug("Election:Leader has following peers:"+ followers);
-
-        if (followers.size() > 0) {
-            minReplicationCount = (followers.size() + 1) / 2 + 1;
-        } else {
-            minReplicationCount = 0;
-        }
-
-
-        // Immediately schedule a heartbeat
-        // Upon election: send initial empty AppendEntries RPCs
-        // (heartbeat) to each server; repeat during idle periods to
-        // prevent election timeouts (§5.2)
-        scheduleHeartBeat(new FiniteDuration(0, TimeUnit.SECONDS));
+public class Leader extends AbstractLeader {
+    /**
+     * 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(
-            new FiniteDuration(context.getConfigParams().getHeartBeatInterval().length() * 1000,
-                context.getConfigParams().getHeartBeatInterval().unit())
-        );
+    private final Stopwatch isolatedLeaderCheck = Stopwatch.createStarted();
+    private @Nullable LeadershipTransferContext leadershipTransferContext;
 
+    Leader(final RaftActorContext context, @Nullable final AbstractLeader initializeFromLeader) {
+        super(context, RaftState.Leader, initializeFromLeader);
     }
 
-    @Override protected RaftState handleAppendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
-
-        context.getLogger().info("Leader: Received {}", appendEntries.toString());
-
-        return state();
+    public Leader(final RaftActorContext context) {
+        this(context, null);
     }
 
-    @Override protected RaftState handleAppendEntriesReply(ActorRef sender,
-        AppendEntriesReply appendEntriesReply) {
-
-        if(! appendEntriesReply.isSuccess()) {
-            context.getLogger()
-                .info("Leader: Received {}", appendEntriesReply.toString());
-        }
-
-        // Update the FollowerLogInformation
-        String followerId = appendEntriesReply.getFollowerId();
-        FollowerLogInformation followerLogInformation =
-            followerToLog.get(followerId);
-
-        if(followerLogInformation == null){
-            context.getLogger().error("Unknown follower {}", followerId);
-            return state();
-        }
-
-        if (appendEntriesReply.isSuccess()) {
-            followerLogInformation
-                .setMatchIndex(appendEntriesReply.getLogLastIndex());
-            followerLogInformation
-                .setNextIndex(appendEntriesReply.getLogLastIndex() + 1);
-        } else {
-
-            // TODO: When we find that the follower is out of sync with the
-            // Leader we simply decrement that followers next index by 1.
-            // Would it be possible to do better than this? The RAFT spec
-            // does not explicitly deal with it but may be something for us to
-            // think about
+    @Override
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object originalMessage) {
+        requireNonNull(sender, "sender should not be null");
 
-            followerLogInformation.decrNextIndex();
-        }
-
-        // Now figure out if this reply warrants a change in the commitIndex
-        // If there exists an N such that N > commitIndex, a majority
-        // of matchIndex[i] ≥ N, and log[N].term == currentTerm:
-        // set commitIndex = N (§5.3, §5.4).
-        for (long N = context.getCommitIndex() + 1; ; N++) {
-            int replicatedCount = 1;
-
-            for (FollowerLogInformation info : followerToLog.values()) {
-                if (info.getMatchIndex().get() >= N) {
-                    replicatedCount++;
-                }
-            }
-
-            if (replicatedCount >= minReplicationCount) {
-                ReplicatedLogEntry replicatedLogEntry =
-                    context.getReplicatedLog().get(N);
-                if (replicatedLogEntry != null
-                    && replicatedLogEntry.getTerm()
-                    == currentTerm()) {
-                    context.setCommitIndex(N);
-                }
+        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(), getLeaderId());
+                return internalSwitchBehavior(new IsolatedLeader(context, this));
             } else {
-                break;
+                return this;
             }
+        } else {
+            return super.handleMessage(sender, originalMessage);
         }
-
-        // Apply the change to the state machine
-        if (context.getCommitIndex() > context.getLastApplied()) {
-            applyLogToStateMachine(context.getCommitIndex());
-        }
-
-        return state();
     }
 
-    protected ClientRequestTracker findClientRequestTracker(long logIndex) {
-        for (ClientRequestTracker tracker : trackerList) {
-            if (tracker.getIndex() == logIndex) {
-                return tracker;
-            }
+    @Override
+    protected void beforeSendHeartbeat() {
+        if (isolatedLeaderCheck.elapsed(TimeUnit.MILLISECONDS)
+                > context.getConfigParams().getIsolatedCheckIntervalInMillis()) {
+            context.getActor().tell(ISOLATED_LEADER_CHECK, context.getActor());
+            isolatedLeaderCheck.reset().start();
         }
 
-        return null;
+        if (leadershipTransferContext != null && leadershipTransferContext.isExpired(
+                context.getConfigParams().getElectionTimeOutInterval().toMillis())) {
+            log.debug("{}: Leadership transfer expired", logName());
+            leadershipTransferContext = null;
+        }
     }
 
-    @Override protected RaftState handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
-        return state();
+    @Override
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+            final AppendEntriesReply appendEntriesReply) {
+        RaftActorBehavior returnBehavior = super.handleAppendEntriesReply(sender, appendEntriesReply);
+        tryToCompleteLeadershipTransfer(appendEntriesReply.getFollowerId());
+        return returnBehavior;
     }
 
-    @Override public RaftState state() {
-        return RaftState.Leader;
-    }
+    /**
+     * Attempts to transfer leadership to a follower as per the raft paper (§3.10) as follows:
+     * <ul>
+     * <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,
+     *   <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></li>
+     * <li>Otherwise if the election time out period elapses, notify
+     *     {@link RaftActorLeadershipTransferCohort#abortTransfer}.</li>
+     * </ul>
+     *
+     * @param leadershipTransferCohort the cohort participating in the leadership transfer
+     */
+    public void transferLeadership(@NonNull final RaftActorLeadershipTransferCohort leadershipTransferCohort) {
+        log.debug("{}: Attempting to transfer leadership", logName());
 
-    @Override public RaftState handleMessage(ActorRef sender, Object originalMessage) {
-        Preconditions.checkNotNull(sender, "sender should not be null");
+        leadershipTransferContext = new LeadershipTransferContext(leadershipTransferCohort);
 
-        Object message = fromSerializableMessage(originalMessage);
+        // Send an immediate heart beat to the followers.
+        sendAppendEntries(0, false);
+    }
 
-        if (message instanceof RaftRPC) {
-            RaftRPC rpc = (RaftRPC) message;
-            // If RPC request or response contains term T > currentTerm:
-            // set currentTerm = T, convert to follower (§5.1)
-            // This applies to all RPC messages and responses
-            if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
-                context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
-                return RaftState.Follower;
-            }
+    private void tryToCompleteLeadershipTransfer(final String followerId) {
+        if (leadershipTransferContext == null) {
+            return;
         }
 
-        try {
-            if (message instanceof SendHeartBeat) {
-                return sendHeartBeat();
-            } else if(message instanceof SendInstallSnapshot) {
-                installSnapshotIfNeeded();
-            } else if (message instanceof Replicate) {
-                replicate((Replicate) message);
-            } else if (message instanceof InstallSnapshotReply){
-                handleInstallSnapshotReply(
-                    (InstallSnapshotReply) message);
-            }
-        } finally {
-            scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
+        final Optional<String> requestedFollowerIdOptional
+                = leadershipTransferContext.transferCohort.getRequestedFollowerId();
+        if (requestedFollowerIdOptional.isPresent() && !requestedFollowerIdOptional.get().equals(followerId)) {
+            // we want to transfer leadership to specific follower
+            return;
         }
 
-        return super.handleMessage(sender, message);
-    }
+        FollowerLogInformation followerInfo = getFollower(followerId);
+        if (followerInfo == null) {
+            return;
+        }
 
-    private void handleInstallSnapshotReply(InstallSnapshotReply message) {
-        InstallSnapshotReply reply = message;
-        String followerId = reply.getFollowerId();
-        FollowerLogInformation followerLogInformation =
-            followerToLog.get(followerId);
+        long lastIndex = context.getReplicatedLog().lastIndex();
+        boolean isVoting = context.getPeerInfo(followerId).isVoting();
 
-        followerLogInformation
-            .setMatchIndex(context.getReplicatedLog().getSnapshotIndex());
-        followerLogInformation
-            .setNextIndex(context.getReplicatedLog().getSnapshotIndex() + 1);
-    }
+        log.debug("{}: tryToCompleteLeadershipTransfer: followerId: {}, matchIndex: {}, lastIndex: {}, isVoting: {}",
+                logName(), followerId, followerInfo.getMatchIndex(), lastIndex, isVoting);
 
-    private void replicate(Replicate replicate) {
-        long logIndex = replicate.getReplicatedLogEntry().getIndex();
+        if (isVoting && followerInfo.getMatchIndex() == lastIndex) {
+            log.debug("{}: Follower's log matches - sending ElectionTimeout", logName());
 
-        context.getLogger().debug("Replicate message " + logIndex);
+            // 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);
 
-        if (followers.size() == 0) {
-            context.setCommitIndex(
-                replicate.getReplicatedLogEntry().getIndex());
+            // Now send a TimeoutNow message to the matching follower to immediately start an election.
+            ActorSelection followerActor = context.getPeerActorSelection(followerId);
+            followerActor.tell(TimeoutNow.INSTANCE, context.getActor());
 
-            context.getActor()
-                .tell(new ApplyState(replicate.getClientActor(),
-                        replicate.getIdentifier(),
-                        replicate.getReplicatedLogEntry()),
-                    context.getActor()
-                );
-        } else {
-
-            // Create a tracker entry we will use this later to notify the
-            // client actor
-            trackerList.add(
-                new ClientRequestTrackerImpl(replicate.getClientActor(),
-                    replicate.getIdentifier(),
-                    logIndex)
-            );
+            log.debug("{}: Leader transfer complete", logName());
 
-            sendAppendEntries();
+            leadershipTransferContext.transferCohort.transferComplete();
+            leadershipTransferContext = null;
         }
     }
 
-    private void sendAppendEntries() {
-        // Send an AppendEntries to all followers
-        for (String followerId : followers) {
-            ActorSelection followerActor =
-                context.getPeerActorSelection(followerId);
-
-            if (followerActor != null) {
-                FollowerLogInformation followerLogInformation =
-                    followerToLog.get(followerId);
-
-                long nextIndex = followerLogInformation.getNextIndex().get();
-
-                List<ReplicatedLogEntry> entries = Collections.emptyList();
-
-                if (context.getReplicatedLog().isPresent(nextIndex)) {
-                    // TODO: Instead of sending all entries from nextIndex
-                    // only send a fixed number of entries to each follower
-                    // This is to avoid the situation where there are a lot of
-                    // entries to install for a fresh follower or to a follower
-                    // that has fallen too far behind with the log but yet is not
-                    // eligible to receive a snapshot
-                    entries =
-                        context.getReplicatedLog().getFrom(nextIndex, 1);
-                }
-
-                followerActor.tell(
-                    new AppendEntries(currentTerm(), context.getId(),
-                        prevLogIndex(nextIndex),
-                        prevLogTerm(nextIndex), entries,
-                        context.getCommitIndex()).toSerializable(),
-                    actor()
-                );
-            }
+    @Override
+    public void close() {
+        if (leadershipTransferContext != null) {
+            LeadershipTransferContext localLeadershipTransferContext = leadershipTransferContext;
+            leadershipTransferContext = null;
+            localLeadershipTransferContext.transferCohort.abortTransfer();
         }
-    }
 
-    /**
-     * An installSnapshot is scheduled at a interval that is a multiple of
-     * a HEARTBEAT_INTERVAL. This is to avoid the need to check for installing
-     * snapshots at every heartbeat.
-     */
-    private void installSnapshotIfNeeded(){
-        for (String followerId : followers) {
-            ActorSelection followerActor =
-                context.getPeerActorSelection(followerId);
-
-            if(followerActor != null) {
-                FollowerLogInformation followerLogInformation =
-                    followerToLog.get(followerId);
-
-                long nextIndex = followerLogInformation.getNextIndex().get();
-
-                if (!context.getReplicatedLog().isPresent(nextIndex) && context
-                    .getReplicatedLog().isInSnapshot(nextIndex)) {
-                    followerActor.tell(
-                        new InstallSnapshot(currentTerm(), context.getId(),
-                            context.getReplicatedLog().getSnapshotIndex(),
-                            context.getReplicatedLog().getSnapshotTerm(),
-                            context.getReplicatedLog().getSnapshot()
-                        ),
-                        actor()
-                    );
-                }
-            }
-        }
+        super.close();
     }
 
-    private RaftState sendHeartBeat() {
-        if (followers.size() > 0) {
-            sendAppendEntries();
-        }
-        return state();
+    @VisibleForTesting
+    void markFollowerActive(final String followerId) {
+        getFollower(followerId).markFollowerActive();
     }
 
-    private void stopHeartBeat() {
-        if (heartbeatSchedule != null && !heartbeatSchedule.isCancelled()) {
-            heartbeatSchedule.cancel();
-        }
+    @VisibleForTesting
+    void markFollowerInActive(final String followerId) {
+        getFollower(followerId).markFollowerInActive();
     }
 
-    private void stopInstallSnapshotSchedule() {
-        if (installSnapshotSchedule != null && !installSnapshotSchedule.isCancelled()) {
-            installSnapshotSchedule.cancel();
-        }
-    }
+    private static class LeadershipTransferContext {
+        RaftActorLeadershipTransferCohort transferCohort;
+        Stopwatch timer = Stopwatch.createStarted();
 
-    private void scheduleHeartBeat(FiniteDuration interval) {
-        if(followers.size() == 0){
-            // Optimization - do not bother scheduling a heartbeat as there are
-            // no followers
-            return;
+        LeadershipTransferContext(final RaftActorLeadershipTransferCohort transferCohort) {
+            this.transferCohort = transferCohort;
         }
 
-        stopHeartBeat();
-
-        // Schedule a heartbeat. When the scheduler triggers a SendHeartbeat
-        // message is sent to itself.
-        // Scheduling the heartbeat only once here because heartbeats do not
-        // need to be sent if there are other messages being sent to the remote
-        // actor.
-        heartbeatSchedule =
-            context.getActorSystem().scheduler().scheduleOnce(
-                interval,
-                context.getActor(), new SendHeartBeat(),
-                context.getActorSystem().dispatcher(), context.getActor());
-    }
-
+        boolean isExpired(final long timeout) {
+            if (timer.elapsed(TimeUnit.MILLISECONDS) >= timeout) {
+                transferCohort.abortTransfer();
+                return true;
+            }
 
-    private void scheduleInstallSnapshotCheck(FiniteDuration interval) {
-        if(followers.size() == 0){
-            // Optimization - do not bother scheduling a heartbeat as there are
-            // no followers
-            return;
+            return false;
         }
-
-        stopInstallSnapshotSchedule();
-
-        // 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 SendInstallSnapshot(),
-                context.getActorSystem().dispatcher(), context.getActor());
-    }
-
-
-
-    @Override public void close() throws Exception {
-        stopHeartBeat();
     }
-
-    @Override public String getLeaderId() {
-        return context.getId();
-    }
-
 }