Implement behavior common to a RaftActor in all it's states
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index fde104223bdafec77dcc5cfbcb8927f0b870fbcb..d7a8d5abb35bdb258633b65a783aed9e0aa9677d 100644 (file)
@@ -8,11 +8,18 @@
 
 package org.opendaylight.controller.cluster.raft.behaviors;
 
+import akka.actor.ActorRef;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
+import org.opendaylight.controller.cluster.raft.RaftState;
+import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
+import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
+import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
+import org.opendaylight.controller.cluster.raft.messages.RequestVote;
+import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 
 /**
  * Abstract class that represents the behavior of a RaftActor
- * <p>
+ * <p/>
  * All Servers:
  * <ul>
  * <li> If commitIndex > lastApplied: increment lastApplied, apply
@@ -31,4 +38,125 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected AbstractRaftActorBehavior(RaftActorContext context) {
         this.context = context;
     }
+
+    /**
+     * Derived classes should not directly handle AppendEntries messages it
+     * should let the base class handle it first. Once the base class handles
+     * the AppendEntries message and does the common actions that are applicable
+     * in all RaftState's it will delegate the handling of the AppendEntries
+     * message to the derived class to do more state specific handling by calling
+     * this method
+     *
+     * @param sender         The actor that sent this message
+     * @param appendEntries  The AppendEntries message
+     * @param suggestedState The state that the RaftActor should be in based
+     *                       on the base class's processing of the AppendEntries
+     *                       message
+     * @return
+     */
+    protected abstract RaftState handleAppendEntries(ActorRef sender,
+        AppendEntries appendEntries, RaftState suggestedState);
+
+    /**
+     * Derived classes should not directly handle AppendEntriesReply messages it
+     * should let the base class handle it first. Once the base class handles
+     * the AppendEntriesReply message and does the common actions that are
+     * applicable in all RaftState's it will delegate the handling of the
+     * AppendEntriesReply message to the derived class to do more state specific
+     * handling by calling this method
+     *
+     * @param sender             The actor that sent this message
+     * @param appendEntriesReply The AppendEntriesReply message
+     * @param suggestedState     The state that the RaftActor should be in based
+     *                           on the base class's processing of the
+     *                           AppendEntriesReply message
+     * @return
+     */
+
+    protected abstract RaftState handleAppendEntriesReply(ActorRef sender,
+        AppendEntriesReply appendEntriesReply, RaftState suggestedState);
+
+    /**
+     * Derived classes should not directly handle RequestVote messages it
+     * should let the base class handle it first. Once the base class handles
+     * the RequestVote message and does the common actions that are applicable
+     * in all RaftState's it will delegate the handling of the RequestVote
+     * message to the derived class to do more state specific handling by calling
+     * this method
+     *
+     * @param sender         The actor that sent this message
+     * @param requestVote    The RequestVote message
+     * @param suggestedState The state that the RaftActor should be in based
+     *                       on the base class's processing of the RequestVote
+     *                       message
+     * @return
+     */
+    protected abstract RaftState handleRequestVote(ActorRef sender,
+        RequestVote requestVote, RaftState suggestedState);
+
+    /**
+     * Derived classes should not directly handle RequestVoteReply messages it
+     * should let the base class handle it first. Once the base class handles
+     * the RequestVoteReply message and does the common actions that are
+     * applicable in all RaftState's it will delegate the handling of the
+     * RequestVoteReply message to the derived class to do more state specific
+     * handling by calling this method
+     *
+     * @param sender           The actor that sent this message
+     * @param requestVoteReply The RequestVoteReply message
+     * @param suggestedState   The state that the RaftActor should be in based
+     *                         on the base class's processing of the RequestVote
+     *                         message
+     * @return
+     */
+
+    protected abstract RaftState handleRequestVoteReply(ActorRef sender,
+        RequestVoteReply requestVoteReply, RaftState suggestedState);
+
+    /**
+     * @return The derived class should return the state that corresponds to
+     * it's behavior
+     */
+    protected abstract RaftState state();
+
+    @Override
+    public RaftState handleMessage(ActorRef sender, Object message) {
+        RaftState raftState = state();
+        if (message instanceof RaftRPC) {
+            raftState = applyTerm((RaftRPC) message);
+        }
+        if (message instanceof AppendEntries) {
+            AppendEntries appendEntries = (AppendEntries) message;
+            if (appendEntries.getLeaderCommit() > context.getLastApplied()
+                .get()) {
+                applyLogToStateMachine(appendEntries.getLeaderCommit());
+            }
+            raftState = handleAppendEntries(sender, appendEntries, raftState);
+        } else if (message instanceof AppendEntriesReply) {
+            raftState =
+                handleAppendEntriesReply(sender, (AppendEntriesReply) message,
+                    raftState);
+        } else if (message instanceof RequestVote) {
+            raftState =
+                handleRequestVote(sender, (RequestVote) message, raftState);
+        } else if (message instanceof RequestVoteReply) {
+            raftState =
+                handleRequestVoteReply(sender, (RequestVoteReply) message,
+                    raftState);
+        }
+        return raftState;
+    }
+
+    private RaftState applyTerm(RaftRPC rpc) {
+        if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()
+            .get()) {
+            context.getTermInformation().update(rpc.getTerm(), null);
+            return RaftState.Follower;
+        }
+        return state();
+    }
+
+    private void applyLogToStateMachine(long index) {
+        context.getLastApplied().set(index);
+    }
 }