Add getPeerIds to RaftActorContext
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
index 0b0b4c7cd642480f92dd600a4f8f10be07977dc4..8cb011f7a71d1b54b76bba86a261612f4f555bb0 100644 (file)
@@ -10,6 +10,7 @@ package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
 import akka.actor.Cancellable;
+import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
@@ -17,7 +18,8 @@ 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.ApplyLogEntries;
+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;
 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
@@ -39,6 +41,8 @@ import scala.concurrent.duration.FiniteDuration;
  */
 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
+    protected static final ElectionTimeout ELECTION_TIMEOUT = new ElectionTimeout();
+
     /**
      * Information about the RaftActor whose behavior this class represents
      */
@@ -59,6 +63,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      */
     protected String leaderId = null;
 
+    private short leaderPayloadVersion = -1;
+
     private long replicatedToAllIndex = -1;
 
     private final String logName;
@@ -102,7 +108,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);
@@ -114,7 +120,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) {
@@ -128,7 +134,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;
         }
@@ -147,7 +153,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);
@@ -158,12 +164,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)
@@ -173,7 +195,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;
 
@@ -187,24 +209,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;
     }
 
     /**
@@ -217,15 +230,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());
@@ -245,7 +257,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     /**
      * schedule a new election
      *
-     * @param interval
+     * @param interval the duration after which we should trigger a new election
      */
     protected void scheduleElection(FiniteDuration interval) {
         stopElection();
@@ -254,67 +266,57 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         // message is sent to itself
         electionCancel =
             context.getActorSystem().scheduler().scheduleOnce(interval,
-                context.getActor(), new ElectionTimeout(),
+                context.getActor(), ELECTION_TIMEOUT,
                 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
+     * @return the client request tracker for the specified logIndex
      */
     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;
@@ -322,9 +324,8 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
 
 
     /**
-     * Find the log index from the previous to last entry in the log
      *
-     * @return
+     * @return log index from the previous to last entry in the log
      */
     protected long prevLogIndex(long index){
         ReplicatedLogEntry prevEntry =
@@ -336,8 +337,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
     }
 
     /**
-     * Find the log term from the previous to last entry in the log
-     * @return
+     * @return log term from the previous to last entry in the log
      */
     protected long prevLogTerm(long index){
         ReplicatedLogEntry prevEntry =
@@ -393,7 +393,7 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         // 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 ApplyLogEntries((int) context.getLastApplied()), actor());
+        actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
     }
 
     protected Object fromSerializableMessage(Object serializable){
@@ -418,17 +418,38 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
         return leaderId;
     }
 
-    protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
-        LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), behavior.state());
+    @Override
+    public short getLeaderPayloadVersion() {
+        return leaderPayloadVersion;
+    }
+
+    public void setLeaderPayloadVersion(short leaderPayloadVersion) {
+        this.leaderPayloadVersion = leaderPayloadVersion;
+    }
+
+    @Override
+    public RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
+        return internalSwitchBehavior(behavior);
+    }
+
+    protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
+        if(context.getRaftPolicy().automaticElectionsEnabled()){
+            return internalSwitchBehavior(newState.createBehavior(context));
+        }
+        return this;
+    }
+
+    private RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
+        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
@@ -460,20 +481,31 @@ public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
      * @param snapshotCapturedIndex
      */
     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
-        //  we would want to keep the lastApplied as its used while capturing snapshots
-        long lastApplied = context.getLastApplied();
-        long tempMin = Math.min(snapshotCapturedIndex, (lastApplied > -1 ? lastApplied - 1 : -1));
-
-        if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin))  {
-            LOG.debug("{}: fakeSnapshot purging log to {} for term {}", logName(), tempMin,
-                    context.getTermInformation().getCurrentTerm());
-
-            //use the term of the temp-min, since we check for isPresent, entry will not be null
-            ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
-            context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
-            context.getReplicatedLog().snapshotCommit();
-            setReplicatedToAllIndex(tempMin);
+        long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex, this);
+
+        if(actualIndex != -1){
+            setReplicatedToAllIndex(actualIndex);
         }
     }
 
+    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);
+        }
+    }
 }