BUG-8618: make sync threshold tuneable
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index 6f107e9ae61ee060071c842810cb82c3e7f92f85..c35de820dba734d8c48ad9493c372c4cf0d510cb 100644 (file)
@@ -18,6 +18,7 @@ import akka.cluster.MemberStatus;
 import akka.japi.Procedure;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Optional;
 import java.util.Set;
@@ -27,7 +28,6 @@ import javax.annotation.Nullable;
 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.Snapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
 import org.opendaylight.controller.cluster.raft.base.messages.TimeoutNow;
@@ -39,6 +39,7 @@ import org.opendaylight.controller.cluster.raft.messages.RaftRPC;
 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
 import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPayload;
+import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
 
 /**
  * The behavior of a RaftActor in the Follower raft state.
@@ -50,8 +51,6 @@ import org.opendaylight.controller.cluster.raft.persisted.ServerConfigurationPay
  * </ul>
  */
 public class Follower extends AbstractRaftActorBehavior {
-    private static final int SYNC_THRESHOLD = 10;
-
     private static final long MAX_ELECTION_TIMEOUT_FACTOR = 18;
 
     private final SyncStatusTracker initialSyncStatusTracker;
@@ -61,16 +60,18 @@ public class Follower extends AbstractRaftActorBehavior {
     private String leaderId;
     private short leaderPayloadVersion;
 
-    public Follower(RaftActorContext context) {
+    public Follower(final RaftActorContext context) {
         this(context, null, (short)-1);
     }
 
-    public Follower(RaftActorContext context, String initialLeaderId, short initialLeaderPayloadVersion) {
+    public Follower(final RaftActorContext context, final String initialLeaderId,
+            final short initialLeaderPayloadVersion) {
         super(context, RaftState.Follower);
         this.leaderId = initialLeaderId;
         this.leaderPayloadVersion = initialLeaderPayloadVersion;
 
-        initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), SYNC_THRESHOLD);
+        initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams()
+            .getSyncIndexThreshold());
 
         if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
             actor().tell(TimeoutNow.INSTANCE, actor());
@@ -95,7 +96,7 @@ public class Follower extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    protected final void setLeaderPayloadVersion(short leaderPayloadVersion) {
+    protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) {
         this.leaderPayloadVersion = leaderPayloadVersion;
     }
 
@@ -107,7 +108,7 @@ public class Follower extends AbstractRaftActorBehavior {
         lastLeaderMessageTimer.start();
     }
 
-    private boolean isLogEntryPresent(long index) {
+    private boolean isLogEntryPresent(final long index) {
         if (context.getReplicatedLog().isInSnapshot(index)) {
             return true;
         }
@@ -117,12 +118,12 @@ public class Follower extends AbstractRaftActorBehavior {
 
     }
 
-    private void updateInitialSyncStatus(long currentLeaderCommit, String newLeaderId) {
+    private void updateInitialSyncStatus(final long currentLeaderCommit, final String newLeaderId) {
         initialSyncStatusTracker.update(newLeaderId, currentLeaderCommit, context.getCommitIndex());
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntries(ActorRef sender, AppendEntries appendEntries) {
+    protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
 
         int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
         if (log.isTraceEnabled()) {
@@ -138,7 +139,7 @@ public class Follower extends AbstractRaftActorBehavior {
         if (snapshotTracker != null && !snapshotTracker.getLeaderId().equals(appendEntries.getLeaderId())) {
             log.debug("{}: snapshot install is in progress but the prior snapshot leaderId {} does not match the "
                 + "AppendEntries leaderId {}", logName(), snapshotTracker.getLeaderId(), appendEntries.getLeaderId());
-            snapshotTracker = null;
+            closeSnapshotTracker();
         }
 
         if (snapshotTracker != null || context.getSnapshotManager().isApplying()) {
@@ -164,11 +165,12 @@ public class Follower extends AbstractRaftActorBehavior {
             // We found that the log was out of sync so just send a negative
             // reply and return
 
-            log.debug("{}: Follower is out-of-sync, so sending negative reply, lastIndex: {}, lastTerm: {}",
-                        logName(), lastIndex, lastTerm());
+            final AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
+                    lastTerm(), context.getPayloadVersion());
 
-            sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
-                    lastTerm(), context.getPayloadVersion()), actor());
+            log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
+
+            sender.tell(reply, actor());
             return this;
         }
 
@@ -205,8 +207,7 @@ public class Follower extends AbstractRaftActorBehavior {
 
                     if (!context.getRaftPolicy().applyModificationToStateBeforeConsensus()) {
 
-                        log.debug("{}: Removing entries from log starting at {}", logName(),
-                                matchEntry.getIndex());
+                        log.info("{}: Removing entries from log starting at {}", logName(), matchEntry.getIndex());
 
                         // Entries do not match so remove all subsequent entries
                         if (!context.getReplicatedLog().removeFromAndPersist(matchEntry.getIndex())) {
@@ -215,7 +216,7 @@ public class Follower extends AbstractRaftActorBehavior {
                             // so we must send back a reply to force a snapshot to completely re-sync the
                             // follower's log and state.
 
-                            log.debug("{}: Could not remove entries - sending reply to force snapshot", logName());
+                            log.info("{}: Could not remove entries - sending reply to force snapshot", logName());
                             sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
                                     lastTerm(), context.getPayloadVersion(), true), actor());
                             return this;
@@ -283,6 +284,18 @@ public class Follower extends AbstractRaftActorBehavior {
             log.debug("{}: Commit index set to {}", logName(), context.getCommitIndex());
         }
 
+        AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
+                lastIndex, lastTerm(), context.getPayloadVersion());
+
+        if (log.isTraceEnabled()) {
+            log.trace("{}: handleAppendEntries returning : {}", logName(), reply);
+        } else if (log.isDebugEnabled() && numLogEntries > 0) {
+            log.debug("{}: handleAppendEntries returning : {}", logName(), reply);
+        }
+
+        // Reply to the leader before applying any previous state so as not to hold up leader consensus.
+        sender.tell(reply, actor());
+
         // If commitIndex > lastApplied: increment lastApplied, apply
         // log[lastApplied] to state machine (§5.3)
         // check if there are any entries to be applied. last-applied can be equal to last-index
@@ -297,17 +310,6 @@ public class Follower extends AbstractRaftActorBehavior {
             applyLogToStateMachine(appendEntries.getLeaderCommit());
         }
 
-        AppendEntriesReply reply = new AppendEntriesReply(context.getId(), currentTerm(), true,
-            lastIndex, lastTerm(), context.getPayloadVersion());
-
-        if (log.isTraceEnabled()) {
-            log.trace("{}: handleAppendEntries returning : {}", logName(), reply);
-        } else if (log.isDebugEnabled() && numLogEntries > 0) {
-            log.debug("{}: handleAppendEntries returning : {}", logName(), reply);
-        }
-
-        sender.tell(reply, actor());
-
         if (!context.getSnapshotManager().isCapturing()) {
             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
         }
@@ -315,7 +317,7 @@ public class Follower extends AbstractRaftActorBehavior {
         return this;
     }
 
-    private boolean isOutOfSync(AppendEntries appendEntries) {
+    private boolean isOutOfSync(final AppendEntries appendEntries) {
 
         long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
         boolean prevEntryPresent = isLogEntryPresent(appendEntries.getPrevLogIndex());
@@ -329,14 +331,14 @@ public class Follower extends AbstractRaftActorBehavior {
             // 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 {}",
+            log.info("{}: 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 - "
+            log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - "
                     + "lastIndex: {}, snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex,
                     context.getReplicatedLog().getSnapshotIndex());
         } else if (lastIndex > -1 && prevEntryPresent && prevLogTerm != appendEntries.getPrevLogTerm()) {
@@ -345,7 +347,7 @@ public class Follower extends AbstractRaftActorBehavior {
             // prevLogIndex entry does exist in the follower's log but it has
             // a different term in it
 
-            log.debug("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append entries"
+            log.info("{}: The prevLogIndex {} was found in the log but the term {} is not equal to the append entries"
                       + "prevLogTerm {} - lastIndex: {}, snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(),
                       prevLogTerm, appendEntries.getPrevLogTerm(), lastIndex,
                       context.getReplicatedLog().getSnapshotIndex());
@@ -355,12 +357,12 @@ public class Follower extends AbstractRaftActorBehavior {
             // 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"
+            log.info("{}: 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)) {
-            log.debug("{}: Cannot append entries because the calculated previousIndex {} was not found in the "
+            log.info("{}: 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;
@@ -369,19 +371,19 @@ public class Follower extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
-        AppendEntriesReply appendEntriesReply) {
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+        final AppendEntriesReply appendEntriesReply) {
         return this;
     }
 
     @Override
-    protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
+    protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
+        final RequestVoteReply requestVoteReply) {
         return this;
     }
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
         if (message instanceof ElectionTimeout || message instanceof TimeoutNow) {
             return handleElectionTimeout(message);
         }
@@ -396,7 +398,7 @@ public class Follower extends AbstractRaftActorBehavior {
         // 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",
+            log.info("{}: Term {} in \"{}\" message is greater than follower's term {} - updating term",
                 logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm());
 
             context.getTermInformation().updateAndPersist(rpc.getTerm(), null);
@@ -417,7 +419,7 @@ public class Follower extends AbstractRaftActorBehavior {
         return super.handleMessage(sender, rpc);
     }
 
-    private RaftActorBehavior handleElectionTimeout(Object message) {
+    private RaftActorBehavior handleElectionTimeout(final Object message) {
         // If the message is ElectionTimeout, verify we haven't actually seen a message from the leader
         // during the election timeout interval. It may that the election timer expired b/c this actor
         // was busy and messages got delayed, in which case leader messages would be backed up in the
@@ -510,14 +512,15 @@ public class Follower extends AbstractRaftActorBehavior {
         return false;
     }
 
-    private void handleInstallSnapshot(final ActorRef sender, InstallSnapshot installSnapshot) {
+    private void handleInstallSnapshot(final ActorRef sender, final InstallSnapshot installSnapshot) {
 
         log.debug("{}: handleInstallSnapshot: {}", logName(), installSnapshot);
 
         leaderId = installSnapshot.getLeaderId();
 
         if (snapshotTracker == null) {
-            snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId());
+            snapshotTracker = new SnapshotTracker(log, installSnapshot.getTotalChunks(), installSnapshot.getLeaderId(),
+                    context);
         }
 
         updateInitialSyncStatus(installSnapshot.getLastIncludedIndex(), installSnapshot.getLeaderId());
@@ -528,7 +531,11 @@ public class Follower extends AbstractRaftActorBehavior {
 
             if (snapshotTracker.addChunk(installSnapshot.getChunkIndex(), installSnapshot.getData(),
                     installSnapshot.getLastChunkHashCode())) {
-                Snapshot snapshot = Snapshot.create(snapshotTracker.getSnapshot(),
+
+                log.info("{}: Snapshot installed from leader: {}", logName(), installSnapshot.getLeaderId());
+
+                Snapshot snapshot = Snapshot.create(
+                        context.getSnapshotManager().convertSnapshot(snapshotTracker.getSnapshotBytes()),
                         new ArrayList<>(),
                         installSnapshot.getLastIncludedIndex(),
                         installSnapshot.getLastIncludedTerm(),
@@ -554,24 +561,32 @@ public class Follower extends AbstractRaftActorBehavior {
 
                 actor().tell(new ApplySnapshot(snapshot, applySnapshotCallback), actor());
 
-                snapshotTracker = null;
+                closeSnapshotTracker();
             } else {
                 log.debug("{}: handleInstallSnapshot returning: {}", logName(), reply);
 
                 sender.tell(reply, actor());
             }
-        } catch (SnapshotTracker.InvalidChunkException e) {
+        } catch (IOException e) {
             log.debug("{}: Exception in InstallSnapshot of follower", logName(), e);
 
             sender.tell(new InstallSnapshotReply(currentTerm(), context.getId(),
                     -1, false), actor());
-            snapshotTracker = null;
 
+            closeSnapshotTracker();
+        }
+    }
+
+    private void closeSnapshotTracker() {
+        if (snapshotTracker != null) {
+            snapshotTracker.close();
+            snapshotTracker = null;
         }
     }
 
     @Override
     public void close() {
+        closeSnapshotTracker();
         stopElection();
     }