BUG-5626: do not allow overriding of RaftActor.handleCommand()
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index c799441d603597ea25d530db8dc6eddc9ced68b6..7e6654d5d6b9849b106a36bf56c76928798d912d 100644 (file)
@@ -9,11 +9,14 @@
 package org.opendaylight.controller.cluster.raft.behaviors;
 
 import akka.actor.ActorRef;
+import akka.japi.Procedure;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import java.util.ArrayList;
 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.ServerConfigurationPayload;
 import org.opendaylight.controller.cluster.raft.Snapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
@@ -22,6 +25,7 @@ import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot;
 import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply;
 import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
+import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 
 /**
@@ -35,13 +39,59 @@ import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
  * </ul>
  */
 public class Follower extends AbstractRaftActorBehavior {
+    private static final int SYNC_THRESHOLD = 10;
+
+    private final SyncStatusTracker initialSyncStatusTracker;
+
+    private final Procedure<ReplicatedLogEntry> appendAndPersistCallback = new Procedure<ReplicatedLogEntry>() {
+        @Override
+        public void apply(ReplicatedLogEntry logEntry) {
+            context.getReplicatedLog().captureSnapshotIfReady(logEntry);
+        }
+    };
 
     private SnapshotTracker snapshotTracker = null;
+    private String leaderId;
+    private short leaderPayloadVersion;
 
     public Follower(RaftActorContext context) {
+        this(context, null, (short)-1);
+    }
+
+    public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) {
         super(context, RaftState.Follower);
+        this.leaderId = initialLeaderId;
+        this.leaderPayloadVersion = initialLeaderPayloadVersion;
+
+        initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
+
+        if(canStartElection()) {
+            if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
+                actor().tell(ElectionTimeout.INSTANCE, actor());
+            } else {
+                scheduleElection(electionDuration());
+            }
+        }
+    }
+
+    @Override
+    public final String getLeaderId() {
+        return leaderId;
+    }
+
+    @VisibleForTesting
+    protected final void setLeaderId(final String leaderId) {
+        this.leaderId = Preconditions.checkNotNull(leaderId);
+    }
 
-        scheduleElection(electionDuration());
+    @Override
+    public short getLeaderPayloadVersion() {
+        return leaderPayloadVersion;
+    }
+
+    @VisibleForTesting
+    protected final void setLeaderPayloadVersion(short leaderPayloadVersion) {
+        this.leaderPayloadVersion = leaderPayloadVersion;
     }
 
     private boolean isLogEntryPresent(long index){
@@ -71,8 +121,12 @@ public class Follower extends AbstractRaftActorBehavior {
         return -1;
     }
 
-    @Override protected RaftActorBehavior handleAppendEntries(ActorRef sender,
-                                                              AppendEntries appendEntries) {
+    private void updateInitialSyncStatus(long currentLeaderCommit, String leaderId){
+        initialSyncStatusTracker.update(leaderId, currentLeaderCommit, context.getCommitIndex());
+    }
+
+    @Override
+    protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) {
 
         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
         if(LOG.isTraceEnabled()) {
@@ -85,52 +139,28 @@ public class Follower extends AbstractRaftActorBehavior {
         // to make it easier to read. Before refactoring ensure tests
         // cover the code properly
 
-        // 1. Reply false if term < currentTerm (§5.1)
-        // This is handled in the appendEntries method of the base class
-
-        // If we got here then we do appear to be talking to the leader
-        leaderId = appendEntries.getLeaderId();
+        if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
+            // if snapshot install is in progress, follower should just acknowledge append entries with a reply.
+            AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
+                    lastIndex(), lastTerm(), context.getPayloadVersion());
 
-        // 2. Reply false if log doesn’t contain an entry at prevLogIndex
-        // whose term matches prevLogTerm (§5.3)
-
-        long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
-        boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
+            if(LOG.isDebugEnabled()) {
+                LOG.debug("{}: snapshot install is in progress, replying immediately with {}", logName(), reply);
+            }
+            sender.tell(reply, actor());
 
+            return this;
+        }
 
-        boolean outOfSync = true;
+        // If we got here then we do appear to be talking to the leader
+        leaderId = appendEntries.getLeaderId();
+        leaderPayloadVersion = appendEntries.getPayloadVersion();
 
+        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
         // First check if the logs are in sync or not
         long lastIndex = lastIndex();
-        if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
-
-            // The follower's log is out of sync because the leader does have
-            // an entry at prevLogIndex and this follower has no entries in
-            // it's log.
-
-            LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
-                        logName(), appendEntries.getPrevLogIndex());
-        } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
-
-            // The follower's log is out of sync because the Leader's
-            // prevLogIndex entry was not found in it's log
-
-            LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
-                        logName(), appendEntries.getPrevLogIndex());
-        } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
-
-            // The follower's log is out of sync because the Leader's
-            // prevLogIndex entry does exist in the follower's log but it has
-            // a different term in it
-
-            LOG.debug(
-                "{}: Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}",
-                 logName(), prevLogTerm, appendEntries.getPrevLogTerm());
-        } else {
-            outOfSync = false;
-        }
 
-        if (outOfSync) {
+        if (isOutOfSync(appendEntries)) {
             // We found that the log was out of sync so just send a negative
             // reply and return
 
@@ -138,7 +168,7 @@ public class Follower extends AbstractRaftActorBehavior {
                         logName(), lastIndex, lastTerm());
 
             sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
-                    lastTerm()), actor());
+                    lastTerm(), context.getPayloadVersion()), actor());
             return this;
         }
 
@@ -153,7 +183,7 @@ public class Follower extends AbstractRaftActorBehavior {
             int addEntriesFrom = 0;
             if (context.getReplicatedLog().size() > 0) {
 
-                // Find the entry up until which the one that is not in the follower's log
+                // Find the entry up until the one that is not in the follower's log
                 for (int i = 0;i < appendEntries.getEntries().size(); i++, addEntriesFrom++) {
                     ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
                     ReplicatedLogEntry newEntry = context.getReplicatedLog().get(matchEntry.getIndex());
@@ -167,12 +197,19 @@ public class Follower extends AbstractRaftActorBehavior {
                         continue;
                     }
 
-                    LOG.debug("{}: Removing entries from log starting at {}", logName(),
+                    if(!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
+
+                        LOG.debug("{}: Removing entries from log starting at {}", logName(),
                                 matchEntry.getIndex());
 
-                    // Entries do not match so remove all subsequent entries
-                    context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
-                    break;
+                        // Entries do not match so remove all subsequent entries
+                        context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex());
+                        break;
+                    } else {
+                        sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+                                lastTerm(), context.getPayloadVersion(), true), actor());
+                        return this;
+                    }
                 }
             }
 
@@ -186,7 +223,11 @@ public class Follower extends AbstractRaftActorBehavior {
 
                 LOG.debug("{}: Append entry to log {}", logName(), entry.getData());
 
-                context.getReplicatedLog().appendAndPersist(entry);
+                context.getReplicatedLog().appendAndPersist(entry, appendAndPersistCallback);
+
+                if(entry.getData() instanceof ServerConfigurationPayload) {
+                    context.updatePeerIds((ServerConfigurationPayload)entry.getData());
+                }
             }
 
             LOG.debug("{}: Log size is now {}", logName(), context.getReplicatedLog().size());
@@ -220,7 +261,7 @@ public class Follower extends AbstractRaftActorBehavior {
         }
 
         AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
-            lastIndex, lastTerm());
+            lastIndex, lastTerm(), context.getPayloadVersion());
 
         if(LOG.isTraceEnabled()) {
             LOG.trace("{}: handleAppendEntries returning : {}", logName(), reply);
@@ -230,66 +271,138 @@ public class Follower extends AbstractRaftActorBehavior {
 
         sender.tell(reply, actor());
 
-        if (!context.isSnapshotCaptureInitiated()) {
+        if (!context.getSnapshotManager().isCapturing()) {
             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
         }
 
         return this;
     }
 
-    @Override protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
+    private boolean isOutOfSync(AppendEntries appendEntries) {
+
+        long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
+        boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
+        long lastIndex = lastIndex();
+        int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
+        boolean outOfSync = true;
+
+        if (lastIndex == -1 && appendEntries.getPrevLogIndex() != -1) {
+
+            // The follower's log is out of sync because the leader does have
+            // an entry at prevLogIndex and this follower has no entries in
+            // it's log.
+
+            LOG.debug("{}: The followers log is empty and the senders prevLogIndex is {}",
+                        logName(), appendEntries.getPrevLogIndex());
+        } else if (lastIndex > -1 && appendEntries.getPrevLogIndex() != -1 && !prevEntryPresent) {
+
+            // The follower's log is out of sync because the Leader's
+            // prevLogIndex entry was not found in it's log
+
+            LOG.debug("{}: The log is not empty but the prevLogIndex {} was not found in it",
+                        logName(), appendEntries.getPrevLogIndex());
+        } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
+
+            // The follower's log is out of sync because the Leader's
+            // prevLogIndex entry does exist in the follower's log but it has
+            // a different term in it
+
+            LOG.debug(
+                    "{}: Cannot append entries because previous entry term {}  is not equal to append entries prevLogTerm {}",
+                    logName(), prevLogTerm, appendEntries.getPrevLogTerm());
+        } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
+                && appendEntries.getReplicatedToAllIndex() != -1
+                && !isLogEntryPresent(appendEntries.getReplicatedToAllIndex())
+                && !context.getReplicatedLog().isInSnapshot(appendEntries.getReplicatedToAllIndex())) {
+            // This append entry comes from a leader who has it's log aggressively trimmed and so does not have
+            // the previous entry in it's in-memory journal
+
+            LOG.debug(
+                    "{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the in-memory journal",
+                    logName(), appendEntries.getReplicatedToAllIndex());
+        } else if(appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
+                && appendEntries.getReplicatedToAllIndex() != -1 && numLogEntries > 0
+                && !isLogEntryPresent(appendEntries.getEntries().get(0).getIndex() - 1)
+                && !context.getReplicatedLog().isInSnapshot(appendEntries.getEntries().get(0).getIndex() - 1)) {
+            LOG.debug(
+                    "{}: Cannot append entries because the calculated previousIndex {} was not found in the in-memory journal",
+                    logName(), appendEntries.getEntries().get(0).getIndex() - 1);
+        } else {
+            outOfSync = false;
+        }
+        return outOfSync;
+    }
+
+    @Override
+    protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
         AppendEntriesReply appendEntriesReply) {
         return this;
     }
 
-    @Override protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
+    @Override
+    protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
         RequestVoteReply requestVoteReply) {
         return this;
     }
 
-    @Override public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
+    @Override
+    public RaftActorBehavior handleMessage(ActorRef sender, Object originalMessage) {
+        if (originalMessage instanceof ElectionTimeout) {
+            if (canStartElection()) {
+                LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
+                return internalSwitchBehavior(RaftState.Candidate);
+            } else {
+                return this;
+            }
+        }
 
-        Object message = fromSerializableMessage(originalMessage);
+        final Object message = fromSerializableMessage(originalMessage);
+        if (!(message instanceof RaftRPC)) {
+            // The rest of the processing requires the message to be a RaftRPC
+            return null;
+        }
 
-        if (message instanceof RaftRPC) {
-            RaftRPC rpc = (RaftRPC) message;
-            // If RPC request or response contains term T > currentTerm:
-            // set currentTerm = T, convert to follower (§5.1)
-            // This applies to all RPC messages and responses
-            if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
-                LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
-                        logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
+        final RaftRPC rpc = (RaftRPC) message;
+        // If RPC request or response contains term T > currentTerm:
+        // set currentTerm = T, convert to follower (§5.1)
+        // This applies to all RPC messages and responses
+        if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) {
+            LOG.debug("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
+                logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
 
-                context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
-            }
+            context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
         }
 
-        if (message instanceof ElectionTimeout) {
-            LOG.debug("{}: Received ElectionTimeout - switching to Candidate", logName());
-            return switchBehavior(new Candidate(context));
-
-        } else if (message instanceof InstallSnapshot) {
-            InstallSnapshot installSnapshot = (InstallSnapshot) message;
+        if (rpc instanceof InstallSnapshot) {
+            InstallSnapshot installSnapshot = (InstallSnapshot) rpc;
             handleInstallSnapshot(sender, installSnapshot);
+            scheduleElection(electionDuration());
+            return this;
         }
 
-        scheduleElection(electionDuration());
+        if (!(rpc instanceof RequestVote) || canGrantVote((RequestVote) rpc)) {
+            scheduleElection(electionDuration());
+        }
 
-        return super.handleMessage(sender, message);
+        return super.handleMessage(sender, rpc);
     }
 
-    private void handleInstallSnapshot(ActorRef sender, InstallSnapshot installSnapshot) {
+    private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
 
+        LOG.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
 
-        LOG.debug("{}: InstallSnapshot received from leader {}, datasize: {} , Chunk: {}/{}",
-                    logName(), installSnapshot.getLeaderId(), installSnapshot.getData().size(),
-                    installSnapshot.getChunkIndex(), installSnapshot.getTotalChunks());
+        leaderId = installSnapshot.getLeaderId();
 
         if(snapshotTracker == null){
             snapshotTracker = new SnapshotTracker(LOG, installSnapshot.getTotalChunks());
         }
 
+        updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
+
         try {
+            final InstallSnapshotReply reply = new InstallSnapshotReply(
+                    currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
+
             if(snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
                     installSnapshot.getLastChunkHashCode())){
                 Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
@@ -297,21 +410,33 @@ public class Follower extends AbstractRaftActorBehavior {
                         installSnapshot.getLastIncludedIndex(),
                         installSnapshot.getLastIncludedTerm(),
                         installSnapshot.getLastIncludedIndex(),
-                        installSnapshot.getLastIncludedTerm());
-
-                actor().tell(new ApplySnapshot(snapshot), actor());
+                        installSnapshot.getLastIncludedTerm(),
+                        context.getTermInformation().getCurrentTerm(),
+                        context.getTermInformation().getVotedFor(),
+                        context.getPeerServerInfo(true));
 
-                snapshotTracker = null;
+                ApplySnapshot.Callback applySnapshotCallback = new ApplySnapshot.Callback() {
+                    @Override
+                    public void onSuccess() {
+                        LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
 
-            }
+                        sender.tell(reply, actor());
+                    }
 
-            InstallSnapshotReply reply = new InstallSnapshotReply(
-                    currentTerm(), context.getId(), installSnapshot.getChunkIndex(), true);
+                    @Override
+                    public void onFailure() {
+                        sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(), -1, false), actor());
+                    }
+                };
 
-            LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
+                actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
 
-            sender.tell(reply, actor());
+                snapshotTracker = null;
+            } else {
+                LOG.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
 
+                sender.tell(reply, actor());
+            }
         } catch (SnapshotTracker.InvalidChunkException e) {
             LOG.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
 
@@ -330,7 +455,7 @@ public class Follower extends AbstractRaftActorBehavior {
     }
 
     @Override
-    public void close() throws Exception {
+    public void close() {
         stopElection();
     }