BUG-5626: use Identifier instead of String
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index 8cb011f7a71d1b54b76bba86a261612f4f555bb0..106678e4742a666617a280f79399f91c73286126 100644 (file)
@@ -10,7 +10,7 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
-import java.util.Map;
+import com.google.common.base.Preconditions;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
@@ -18,7 +18,6 @@ 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.ServerConfigurationPayload;
 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;
@@ -26,6 +25,7 @@ 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.yangtools.concepts.Identifier;
 import org.slf4j.Logger;
 import scala.concurrent.duration.FiniteDuration;
 
@@ -40,9 +40,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
      */
@@ -58,33 +55,41 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      */
     private Cancellable electionCancel = null;
 
-    /**
-     *
-     */
-    protected String leaderId = null;
-
-    private short leaderPayloadVersion = -1;
-
     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);
+            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;
     }
 
@@ -254,6 +259,10 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         }
     }
 
+    protected boolean canStartElection() {
+        return context.getRaftPolicy().automaticElectionsEnabled() && context.isVotingMember();
+    }
+
     /**
      * schedule a new election
      *
@@ -262,12 +271,11 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     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());
+        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());
+        }
     }
 
     /**
@@ -356,30 +364,28 @@ 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);
-
+        for (long i = context.getLastApplied() + 1; i < index + 1; i++) {
+            final ActorRef clientActor;
+            final Identifier identifier;
+            final ClientRequestTracker tracker = removeClientRequestTracker(i);
             if (tracker != null) {
                 clientActor = tracker.getClientActor();
                 identifier = tracker.getIdentifier();
+            } else {
+                clientActor = null;
+                identifier = null;
             }
-            ReplicatedLogEntry replicatedLogEntry =
-                context.getReplicatedLog().get(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());
+                actor().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), 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;
             }
@@ -410,21 +416,9 @@ 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 short getLeaderPayloadVersion() {
-        return leaderPayloadVersion;
-    }
-
-    public void setLeaderPayloadVersion(short leaderPayloadVersion) {
-        this.leaderPayloadVersion = leaderPayloadVersion;
     }
 
     @Override
@@ -434,7 +428,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
     protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
         if(context.getRaftPolicy().automaticElectionsEnabled()){
-            return internalSwitchBehavior(newState.createBehavior(context));
+            return internalSwitchBehavior(createBehavior(context, newState));
         }
         return this;
     }
@@ -481,7 +475,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);
@@ -491,21 +485,4 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     protected String getId(){
         return context.getId();
     }
-
-    public void applyServerConfiguration(ServerConfigurationPayload serverConfig) {
-        Map<String, String> currentPeers = context.getPeerAddresses();
-        for(String peerId: serverConfig.getNewServerConfig()) {
-            if(!getId().equals(peerId)) {
-                if(!currentPeers.containsKey(peerId)) {
-                    context.addToPeers(peerId, null);
-                } else {
-                    currentPeers.remove(peerId);
-                }
-            }
-        }
-
-        for(String peerIdToRemove: currentPeers.keySet()) {
-            context.removePeer(peerIdToRemove);
-        }
-    }
 }