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 98cbd7b38179710e59733a0b08a2e53692e84f29..375b3779b891dd94e1a3b702f9396853478de265 100644 (file)
@@ -17,14 +17,13 @@ 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;
 
@@ -78,6 +77,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
                 return new IsolatedLeader(context);
             case Leader:
                 return new Leader(context);
+            case PreLeader:
+                return new PreLeader(context);
             default:
                 throw new IllegalArgumentException("Unhandled state " + state);
         }
@@ -270,11 +271,9 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected void scheduleElection(FiniteDuration interval) {
         stopElection();
 
-        if(canStartElection()) {
-            // 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());
-        }
+        // 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());
     }
 
     /**
@@ -323,26 +322,34 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
     /**
      *
-     * @return log index from the previous to last entry in the log
+     * @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;
     }
 
     /**
-     * @return log term from the previous to last entry in the log
+     * @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;
     }
 
@@ -391,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) {
@@ -416,13 +419,14 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
-        if(context.getRaftPolicy().automaticElectionsEnabled()){
-            return internalSwitchBehavior(createBehavior(context, newState));
-        }
-        return this;
+        return internalSwitchBehavior(createBehavior(context, newState));
     }
 
-    private RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
+    protected RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
+        if(!context.getRaftPolicy().automaticElectionsEnabled()) {
+            return this;
+        }
+
         LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
         try {
             close();