Further Guava Optional cleanups
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index 0a553b40fd59aab555f258f897a2154830afd1c8..087c656b1836daaf9c05ac7b91e32eba1057c02c 100644 (file)
@@ -10,52 +10,101 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
+import com.google.common.base.Preconditions;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
 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.SerializationUtils;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
+import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
+import org.slf4j.Logger;
 import scala.concurrent.duration.FiniteDuration;
 
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
 /**
- * Abstract class that represents the behavior of a RaftActor
- * <p/>
- * All Servers:
- * <ul>
- * <li> If commitIndex > lastApplied: increment lastApplied, apply
- * log[lastApplied] to state machine (§5.3)
- * <li> If RPC request or response contains term T > currentTerm:
- * set currentTerm = T, convert to follower (§5.1)
+ * Abstract class that provides common code for a RaftActor behavior.
  */
 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
-
     /**
-     * Information about the RaftActor whose behavior this class represents
+     * Information about the RaftActor whose behavior this class represents.
      */
     protected final RaftActorContext context;
 
     /**
-     *
+     * Used for message logging.
+     */
+    @SuppressFBWarnings("SLF4J_LOGGER_SHOULD_BE_PRIVATE")
+    protected final Logger log;
+
+    /**
+     * Prepended to log messages to provide appropriate context.
+     */
+    private final String logName;
+
+    /**
+     * The RaftState corresponding to his behavior.
+     */
+    private final RaftState state;
+
+    /**
+     * Used to cancel a scheduled election.
      */
     private Cancellable electionCancel = null;
 
     /**
-     *
+     * The index of the last log entry that has been replicated to all raft peers.
      */
-    protected String leaderId = null;
+    private long replicatedToAllIndex = -1;
+
+    AbstractRaftActorBehavior(final RaftActorContext context, final RaftState state) {
+        this.context = Preconditions.checkNotNull(context);
+        this.state = Preconditions.checkNotNull(state);
+        this.log = context.getLogger();
+
+        logName = String.format("%s (%s)", context.getId(), state);
+    }
+
+    public static RaftActorBehavior createBehavior(final RaftActorContext context, final RaftState state) {
+        switch (state) {
+            case Candidate:
+                return new Candidate(context);
+            case Follower:
+                return new Follower(context);
+            case IsolatedLeader:
+                return new IsolatedLeader(context);
+            case Leader:
+                return new Leader(context);
+            case PreLeader:
+                return new PreLeader(context);
+            default:
+                throw new IllegalArgumentException("Unhandled state " + state);
+        }
+    }
 
+    @Override
+    public final RaftState state() {
+        return state;
+    }
+
+    protected final String logName() {
+        return logName;
+    }
+
+    @Override
+    public void setReplicatedToAllIndex(final long replicatedToAllIndex) {
+        this.replicatedToAllIndex = replicatedToAllIndex;
+    }
 
-    protected AbstractRaftActorBehavior(RaftActorContext context) {
-        this.context = context;
+    @Override
+    public long getReplicatedToAllIndex() {
+        return replicatedToAllIndex;
     }
 
     /**
@@ -68,33 +117,28 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      *
      * @param sender         The actor that sent this message
      * @param appendEntries  The AppendEntries message
-     * @return
+     * @return a new behavior if it was changed or the current behavior
      */
-    protected abstract RaftState handleAppendEntries(ActorRef sender,
+    protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries);
 
-
     /**
-     * appendEntries first processes the AppendEntries message and then
-     * delegates handling to a specific behavior
+     * Handles the common logic for the AppendEntries message and delegates handling to the derived class.
      *
-     * @param sender
-     * @param appendEntries
-     * @return
+     * @param sender the ActorRef that sent the message
+     * @param appendEntries the message
+     * @return a new behavior if it was changed or the current behavior
      */
-    protected RaftState appendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
+    protected RaftActorBehavior appendEntries(final ActorRef sender, final AppendEntries appendEntries) {
 
         // 1. Reply false if term < currentTerm (§5.1)
         if (appendEntries.getTerm() < currentTerm()) {
-            context.getLogger().debug(
-                "Cannot append entries because sender term " + appendEntries
-                    .getTerm() + " is less than " + currentTerm());
-            sender.tell(
-                new AppendEntriesReply(context.getId(), currentTerm(), false,
-                    lastIndex(), lastTerm()), actor()
-            );
-            return state();
+            log.info("{}: Cannot append entries because sender's term {} is less than {}", logName(),
+                    appendEntries.getTerm(), currentTerm());
+
+            sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(), lastTerm(),
+                    context.getPayloadVersion(), false, false, appendEntries.getLeaderRaftVersion()), actor());
+            return this;
         }
 
 
@@ -111,22 +155,39 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      *
      * @param sender             The actor that sent this message
      * @param appendEntriesReply The AppendEntriesReply message
-     * @return
+     * @return a new behavior if it was changed or the current behavior
      */
-    protected abstract RaftState handleAppendEntriesReply(ActorRef sender,
+    protected abstract RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply);
 
     /**
-     * requestVote handles the RequestVote message. This logic is common
-     * for all behaviors
+     * Handles the logic for the RequestVote message that is common for all behaviors.
      *
-     * @param sender
-     * @param requestVote
-     * @return
+     * @param sender the ActorRef that sent the message
+     * @param requestVote the message
+     * @return a new behavior if it was changed or the current behavior
      */
-    protected RaftState requestVote(ActorRef sender,
-        RequestVote requestVote) {
+    protected RaftActorBehavior requestVote(final ActorRef sender, final RequestVote requestVote) {
+
+        log.debug("{}: In requestVote:  {} - currentTerm: {}, votedFor: {}, lastIndex: {}, lastTerm: {}", logName(),
+                requestVote, currentTerm(), votedFor(), lastIndex(), lastTerm());
+
+        boolean grantVote = canGrantVote(requestVote);
 
+        if (grantVote) {
+            context.getTermInformation().updateAndPersist(requestVote.getTerm(), requestVote.getCandidateId());
+        }
+
+        RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote);
+
+        log.debug("{}: requestVote returning: {}", logName(), reply);
+
+        sender.tell(reply, actor());
+
+        return this;
+    }
+
+    protected boolean canGrantVote(final RequestVote requestVote) {
         boolean grantVote = false;
 
         //  Reply false if term < currentTerm (§5.1)
@@ -136,7 +197,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             // If votedFor is null or candidateId, and candidate’s log is at
             // least as up-to-date as receiver’s log, grant vote (§5.2, §5.4)
         } else if (votedFor() == null || votedFor()
-            .equals(requestVote.getCandidateId())) {
+                .equals(requestVote.getCandidateId())) {
 
             boolean candidateLatest = false;
 
@@ -149,21 +210,16 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             // more up-to-date.
             if (requestVote.getLastLogTerm() > lastTerm()) {
                 candidateLatest = true;
-            } else if ((requestVote.getLastLogTerm() == lastTerm())
-                && requestVote.getLastLogIndex() >= lastIndex()) {
+            } else if (requestVote.getLastLogTerm() == lastTerm()
+                    && requestVote.getLastLogIndex() >= lastIndex()) {
                 candidateLatest = true;
             }
 
             if (candidateLatest) {
                 grantVote = true;
-                context.getTermInformation().updateAndPersist(requestVote.getTerm(),
-                    requestVote.getCandidateId());
             }
         }
-
-        sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor());
-
-        return state();
+        return grantVote;
     }
 
     /**
@@ -176,24 +232,24 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      *
      * @param sender           The actor that sent this message
      * @param requestVoteReply The RequestVoteReply message
-     * @return
+     * @return a new behavior if it was changed or the current behavior
      */
-    protected abstract RaftState handleRequestVoteReply(ActorRef sender,
+    protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender,
         RequestVoteReply requestVoteReply);
 
     /**
-     * Creates a random election duration
+     * Returns a duration for election with an additional variance for randomness.
      *
-     * @return
+     * @return a random election duration
      */
     protected FiniteDuration electionDuration() {
         long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
         return context.getConfigParams().getElectionTimeOutInterval().$plus(
-            new FiniteDuration(variance, TimeUnit.MILLISECONDS));
+                new FiniteDuration(variance, TimeUnit.MILLISECONDS));
     }
 
     /**
-     * stop the scheduled election
+     * Stops the currently scheduled election.
      */
     protected void stopElection() {
         if (electionCancel != null && !electionCancel.isCancelled()) {
@@ -201,140 +257,171 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         }
     }
 
+    protected boolean canStartElection() {
+        return context.getRaftPolicy().automaticElectionsEnabled() && context.isVotingMember();
+    }
+
     /**
-     * schedule a new election
+     * Schedule a new election.
      *
-     * @param interval
+     * @param interval the duration after which we should trigger a new election
      */
-    protected void scheduleElection(FiniteDuration interval) {
+    protected void scheduleElection(final FiniteDuration interval) {
         stopElection();
 
-        // Schedule an election. When the scheduler triggers an ElectionTimeout
-        // message is sent to itself
-        electionCancel =
-            context.getActorSystem().scheduler().scheduleOnce(interval,
-                context.getActor(), new ElectionTimeout(),
-                context.getActorSystem().dispatcher(), context.getActor());
+        // Schedule an election. When the scheduler triggers an ElectionTimeout message is sent to itself
+        electionCancel = context.getActorSystem().scheduler().scheduleOnce(interval, context.getActor(),
+                ElectionTimeout.INSTANCE, context.getActorSystem().dispatcher(), context.getActor());
     }
 
     /**
-     * Get the current term
-     * @return
+     * Returns the current election term.
+     *
+     * @return the current term
      */
     protected long currentTerm() {
         return context.getTermInformation().getCurrentTerm();
     }
 
     /**
-     * Get the candidate for whom we voted in the current term
-     * @return
+     * Returns the id of the candidate that this server voted for in current term.
+     *
+     * @return the candidate for whom we voted in the current term
      */
     protected String votedFor() {
         return context.getTermInformation().getVotedFor();
     }
 
     /**
-     * Get the actor associated with this behavior
-     * @return
+     * Returns the actor associated with this behavior.
+     *
+     * @return the actor
      */
     protected ActorRef actor() {
         return context.getActor();
     }
 
     /**
-     * Get the term from the last entry in the log
+     * Returns the term of the last entry in the log.
      *
-     * @return
+     * @return the term
      */
     protected long lastTerm() {
         return context.getReplicatedLog().lastTerm();
     }
 
     /**
-     * Get the index from the last entry in the log
+     * Returns the index of the last entry in the log.
      *
-     * @return
+     * @return the index
      */
     protected long lastIndex() {
         return context.getReplicatedLog().lastIndex();
     }
 
     /**
-     * Find the client request tracker for a specific logIndex
-     *
-     * @param logIndex
-     * @return
+     * Removes and returns the ClientRequestTracker for the specified log index.
+     * @param logIndex the log index
+     * @return the ClientRequestTracker or null if none available
      */
-    protected ClientRequestTracker findClientRequestTracker(long logIndex) {
+    protected ClientRequestTracker removeClientRequestTracker(final long logIndex) {
         return null;
     }
 
     /**
-     * Find the log index from the previous to last entry in the log
+     * Returns the actual index of the entry in replicated log for the given index or -1 if not found.
      *
-     * @return
+     * @return the log entry index or -1 if not found
      */
-    protected long prevLogIndex(long index){
-        ReplicatedLogEntry prevEntry =
-            context.getReplicatedLog().get(index - 1);
-        if (prevEntry != null) {
-            return prevEntry.getIndex();
+    protected long getLogEntryIndex(final long index) {
+        if (index == context.getReplicatedLog().getSnapshotIndex()) {
+            return context.getReplicatedLog().getSnapshotIndex();
+        }
+
+        ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
+        if (entry != null) {
+            return entry.getIndex();
         }
+
         return -1;
     }
 
     /**
-     * Find the log term from the previous to last entry in the log
-     * @return
+     * Returns the actual term of the entry in the replicated log for the given index or -1 if not found.
+     *
+     * @return the log entry term or -1 if not found
      */
-    protected long prevLogTerm(long index){
-        ReplicatedLogEntry prevEntry =
-            context.getReplicatedLog().get(index - 1);
-        if (prevEntry != null) {
-            return prevEntry.getTerm();
+    protected long getLogEntryTerm(final long index) {
+        if (index == context.getReplicatedLog().getSnapshotIndex()) {
+            return context.getReplicatedLog().getSnapshotTerm();
+        }
+
+        ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
+        if (entry != null) {
+            return entry.getTerm();
         }
+
         return -1;
     }
 
     /**
-     * Apply the provided index to the state machine
+     * Returns the actual term of the entry in the replicated log for the given index or, if not present, returns the
+     * snapshot term if the given index is in the snapshot or -1 otherwise.
      *
-     * @param index a log index that is known to be committed
+     * @return the term or -1 otherwise
      */
-    protected void applyLogToStateMachine(long index) {
+    protected long getLogEntryOrSnapshotTerm(final long index) {
+        if (context.getReplicatedLog().isInSnapshot(index)) {
+            return context.getReplicatedLog().getSnapshotTerm();
+        }
+
+        return getLogEntryTerm(index);
+    }
+
+    /**
+     * Applies the log entries up to the specified index that is known to be committed to the state machine.
+     *
+     * @param index the log index
+     */
+    protected void applyLogToStateMachine(final long index) {
         // Now maybe we apply to the state machine
-        for (long i = context.getLastApplied() + 1;
-             i < index + 1; i++) {
-            ActorRef clientActor = null;
-            String identifier = null;
-            ClientRequestTracker tracker = findClientRequestTracker(i);
-
-            if (tracker != null) {
-                clientActor = tracker.getClientActor();
-                identifier = tracker.getIdentifier();
-            }
-            ReplicatedLogEntry replicatedLogEntry =
-                context.getReplicatedLog().get(i);
+        for (long i = context.getLastApplied() + 1; i < index + 1; i++) {
 
+            ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(i);
             if (replicatedLogEntry != null) {
-                actor().tell(new ApplyState(clientActor, identifier,
-                    replicatedLogEntry), actor());
+                // Send a local message to the local RaftActor (it's derived class to be
+                // specific to apply the log to it's index)
+
+                final ApplyState applyState;
+                final ClientRequestTracker tracker = removeClientRequestTracker(i);
+                if (tracker != null) {
+                    applyState = new ApplyState(tracker.getClientActor(), tracker.getIdentifier(), replicatedLogEntry);
+                } else {
+                    applyState = new ApplyState(null, null, replicatedLogEntry);
+                }
+
+                log.debug("{}: Setting last applied to {}", logName(), i);
+
+                context.setLastApplied(i);
+                context.getApplyStateConsumer().accept(applyState);
             } else {
-                context.getLogger().error(
-                    "Missing index " + i + " from log. Cannot apply state.");
+                //if one index is not present in the log, no point in looping
+                // around as the rest wont be present either
+                log.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
+                        logName(), i, i, index);
+                break;
             }
         }
-        // Send a local message to the local RaftActor (it's derived class to be
-        // specific to apply the log to it's index)
-        context.setLastApplied(index);
-    }
 
-    protected Object fromSerializableMessage(Object serializable){
-        return SerializationUtils.fromSerializable(serializable);
+        // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
+        // will be used during recovery
+        //in case if the above code throws an error and this message is not sent, it would be fine
+        // as the  append entries received later would initiate add this message to the journal
+        actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
     }
 
     @Override
-    public RaftState handleMessage(ActorRef sender, Object message) {
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
         if (message instanceof AppendEntries) {
             return appendEntries(sender, (AppendEntries) message);
         } else if (message instanceof AppendEntriesReply) {
@@ -343,11 +430,76 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             return requestVote(sender, (RequestVote) message);
         } else if (message instanceof RequestVoteReply) {
             return handleRequestVoteReply(sender, (RequestVoteReply) message);
+        } else {
+            return null;
+        }
+    }
+
+    @Override
+    public RaftActorBehavior switchBehavior(final RaftActorBehavior behavior) {
+        return internalSwitchBehavior(behavior);
+    }
+
+    protected RaftActorBehavior internalSwitchBehavior(final RaftState newState) {
+        return internalSwitchBehavior(createBehavior(context, newState));
+    }
+
+    @SuppressWarnings("checkstyle:IllegalCatch")
+    protected RaftActorBehavior internalSwitchBehavior(final RaftActorBehavior newBehavior) {
+        if (!context.getRaftPolicy().automaticElectionsEnabled()) {
+            return this;
+        }
+
+        log.info("{} :- Switching from behavior {} to {}, election term: {}", logName(), this.state(),
+                newBehavior.state(), context.getTermInformation().getCurrentTerm());
+        try {
+            close();
+        } catch (RuntimeException e) {
+            log.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
+        }
+        return newBehavior;
+    }
+
+
+    protected int getMajorityVoteCount(final int numPeers) {
+        // Votes are required from a majority of the peers including self.
+        // The numMajority field therefore stores a calculated value
+        // of the number of votes required for this candidate to win an
+        // election based on it's known peers.
+        // If a peer was added during normal operation and raft replicas
+        // came to know about them then the new peer would also need to be
+        // taken into consideration when calculating this value.
+        // Here are some examples for what the numMajority would be for n
+        // peers
+        // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
+        // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
+        // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
+
+        int numMajority = 0;
+        if (numPeers > 0) {
+            int self = 1;
+            numMajority = (numPeers + self) / 2 + 1;
+        }
+        return numMajority;
+
+    }
+
+
+    /**
+     * Performs a snapshot with no capture on the replicated log. It clears the log from the supplied index or
+     * lastApplied-1 which ever is minimum.
+     *
+     * @param snapshotCapturedIndex the index from which to clear
+     */
+    protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
+        long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
+
+        if (actualIndex != -1) {
+            setReplicatedToAllIndex(actualIndex);
         }
-        return state();
     }
 
-    @Override public String getLeaderId() {
-        return leaderId;
+    protected String getId() {
+        return context.getId();
     }
 }