Bug 6587: Retain state when transitioning between Leader and IsolatedLeader
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index c276d32cce33d5b5bfada40f7f62afb6244a2e07..375b3779b891dd94e1a3b702f9396853478de265 100644 (file)
@@ -10,20 +10,20 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
+import com.google.common.base.Preconditions;
 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.ApplyJournalEntries;
 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;
 
@@ -38,9 +38,6 @@ import scala.concurrent.duration.FiniteDuration;
  * set currentTerm = T, convert to follower (§5.1)
  */
 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
-
-    protected static final ElectionTimeout ELECTION_TIMEOUT = new ElectionTimeout();
-
     /**
      * Information about the RaftActor whose behavior this class represents
      */
@@ -56,31 +53,43 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      */
     private Cancellable electionCancel = null;
 
-    /**
-     *
-     */
-    protected String leaderId = null;
-
     private long replicatedToAllIndex = -1;
 
     private final String logName;
 
     private final RaftState state;
 
-    protected AbstractRaftActorBehavior(RaftActorContext context, RaftState state) {
-        this.context = context;
-        this.state = state;
+    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 RaftState state() {
+    public final RaftState state() {
         return state;
     }
 
-    public String logName() {
+    protected final String logName() {
         return logName;
     }
 
@@ -104,7 +113,7 @@ 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 RaftActorBehavior handleAppendEntries(ActorRef sender,
         AppendEntries appendEntries);
@@ -116,7 +125,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      *
      * @param sender
      * @param appendEntries
-     * @return
+     * @return a new behavior if it was changed or the current behavior
      */
     protected RaftActorBehavior appendEntries(ActorRef sender,
         AppendEntries appendEntries) {
@@ -130,7 +139,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
             sender.tell(
                 new AppendEntriesReply(context.getId(), currentTerm(), false,
-                    lastIndex(), lastTerm()), actor()
+                    lastIndex(), lastTerm(), context.getPayloadVersion()), actor()
             );
             return this;
         }
@@ -149,7 +158,7 @@ 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 RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply);
@@ -160,12 +169,28 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      *
      * @param sender
      * @param requestVote
-     * @return
+     * @return a new behavior if it was changed or the current behavior
      */
     protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) {
 
         LOG.debug("{}: In requestVote:  {}", logName(), requestVote);
 
+        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(RequestVote requestVote){
         boolean grantVote = false;
 
         //  Reply false if term < currentTerm (§5.1)
@@ -175,7 +200,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;
 
@@ -189,24 +214,15 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             if (requestVote.getLastLogTerm() > lastTerm()) {
                 candidateLatest = true;
             } else if ((requestVote.getLastLogTerm() == lastTerm())
-                && requestVote.getLastLogIndex() >= lastIndex()) {
+                    && requestVote.getLastLogIndex() >= lastIndex()) {
                 candidateLatest = true;
             }
 
             if (candidateLatest) {
                 grantVote = true;
-                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;
+        return grantVote;
     }
 
     /**
@@ -219,15 +235,14 @@ 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 RaftActorBehavior handleRequestVoteReply(ActorRef sender,
         RequestVoteReply requestVoteReply);
 
     /**
-     * Creates a random election duration
      *
-     * @return
+     * @return a random election duration
      */
     protected FiniteDuration electionDuration() {
         long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
@@ -244,109 +259,97 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         }
     }
 
+    protected boolean canStartElection() {
+        return context.getRaftPolicy().automaticElectionsEnabled() && context.isVotingMember();
+    }
+
     /**
      * schedule a new election
      *
-     * @param interval
+     * @param interval the duration after which we should trigger a new election
      */
     protected void scheduleElection(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(), ELECTION_TIMEOUT,
-                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
+     * @return the current term
      */
     protected long currentTerm() {
         return context.getTermInformation().getCurrentTerm();
     }
 
     /**
-     * Get the candidate for whom we voted in the current term
-     * @return
+     * @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
+     * @return the actor associated with this behavior
      */
     protected ActorRef actor() {
         return context.getActor();
     }
 
     /**
-     * Get the term from the last entry in the log
      *
-     * @return
+     * @return the term from the last entry in the log
      */
     protected long lastTerm() {
         return context.getReplicatedLog().lastTerm();
     }
 
     /**
-     * Get the index from the last entry in the log
-     *
-     * @return
+     * @return the index from the last entry in the log
      */
     protected long lastIndex() {
         return context.getReplicatedLog().lastIndex();
     }
 
     /**
-     * Find the client request tracker for a specific logIndex
-     *
-     * @param logIndex
-     * @return
-     */
-    protected ClientRequestTracker findClientRequestTracker(long logIndex) {
-        return null;
-    }
-
-    /**
-     * Find the client request tracker for a specific logIndex
-     *
      * @param logIndex
-     * @return
+     * @return the client request tracker for the specified logIndex
      */
     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
         return null;
     }
 
-
     /**
-     * Find the log index from the previous to last entry in the log
      *
-     * @return
+     * @return the log entry index for the given 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(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
+     * @return the log entry term for the given index 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(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;
     }
 
@@ -358,30 +361,27 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected void applyLogToStateMachine(final long index) {
         long newLastApplied = context.getLastApplied();
         // 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 = removeClientRequestTracker(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) {
                 // Send a local message to the local RaftActor (it's derived class to be
                 // specific to apply the log to it's index)
-                actor().tell(new ApplyState(clientActor, identifier,
-                    replicatedLogEntry), actor());
+
+                final ApplyState msg;
+                final ClientRequestTracker tracker = removeClientRequestTracker(i);
+                if (tracker != null) {
+                    msg = new ApplyState(tracker.getClientActor(), tracker.getIdentifier(), replicatedLogEntry);
+                } else {
+                    msg = new ApplyState(null, null, replicatedLogEntry);
+                }
+
+                actor().tell(msg, actor());
                 newLastApplied = i;
             } else {
                 //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 {}",
+                LOG.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
                         logName(), i, i, index);
                 break;
             }
@@ -398,10 +398,6 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
     }
 
-    protected Object fromSerializableMessage(Object serializable){
-        return SerializationUtils.fromSerializable(serializable);
-    }
-
     @Override
     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
         if (message instanceof AppendEntries) {
@@ -412,25 +408,35 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
             return requestVote(sender, (RequestVote) message);
         } else if (message instanceof RequestVoteReply) {
             return handleRequestVoteReply(sender, (RequestVoteReply) message);
+        } else {
+            return null;
         }
-        return this;
     }
 
-    @Override public String getLeaderId() {
-        return leaderId;
+    @Override
+    public RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
+        return internalSwitchBehavior(behavior);
     }
 
-    protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
-        LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), behavior.state());
+    protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
+        return internalSwitchBehavior(createBehavior(context, newState));
+    }
+
+    protected RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
+        if(!context.getRaftPolicy().automaticElectionsEnabled()) {
+            return this;
+        }
+
+        LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
         try {
             close();
         } catch (Exception e) {
             LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
         }
-
-        return behavior;
+        return newBehavior;
     }
 
+
     protected int getMajorityVoteCount(int numPeers) {
         // Votes are required from a majority of the peers including self.
         // The numMajority field therefore stores a calculated value
@@ -462,7 +468,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      * @param snapshotCapturedIndex
      */
     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
-        long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex, this);
+        long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
 
         if(actualIndex != -1){
             setReplicatedToAllIndex(actualIndex);
@@ -472,5 +478,4 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected String getId(){
         return context.getId();
     }
-
 }