Optimize Follower.isOutOfSync()
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index b512089692eed80da136bc8f7dfc2f5a2e10274c..8aee8c1af88c31da507bede83065d3d457b76428 100644 (file)
@@ -20,6 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Stopwatch;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.List;
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
@@ -51,8 +52,6 @@ import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
  * </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;
@@ -62,16 +61,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());
@@ -96,7 +97,7 @@ public class Follower extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    protected final void setLeaderPayloadVersion(short leaderPayloadVersion) {
+    protected final void setLeaderPayloadVersion(final short leaderPayloadVersion) {
         this.leaderPayloadVersion = leaderPayloadVersion;
     }
 
@@ -108,7 +109,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;
         }
@@ -118,12 +119,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()) {
@@ -157,7 +158,6 @@ public class Follower extends AbstractRaftActorBehavior {
         leaderId = appendEntries.getLeaderId();
         leaderPayloadVersion = appendEntries.getPayloadVersion();
 
-        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
         // First check if the logs are in sync or not
         long lastIndex = lastIndex();
 
@@ -169,7 +169,7 @@ public class Follower extends AbstractRaftActorBehavior {
                     lastTerm(), context.getPayloadVersion());
 
             log.info("{}: Follower is out-of-sync so sending negative reply: {}", logName(), reply);
-
+            updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
             sender.tell(reply, actor());
             return this;
         }
@@ -217,6 +217,7 @@ public class Follower extends AbstractRaftActorBehavior {
                             // follower's log and state.
 
                             log.info("{}: Could not remove entries - sending reply to force snapshot", logName());
+                            updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
                             sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
                                     lastTerm(), context.getPayloadVersion(), true), actor());
                             return this;
@@ -224,6 +225,7 @@ public class Follower extends AbstractRaftActorBehavior {
 
                         break;
                     } else {
+                        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
                         sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex,
                                 lastTerm(), context.getPayloadVersion(), true), actor());
                         return this;
@@ -294,6 +296,7 @@ public class Follower extends AbstractRaftActorBehavior {
         }
 
         // Reply to the leader before applying any previous state so as not to hold up leader consensus.
+        updateInitialSyncStatus(appendEntries.getLeaderCommit(), appendEntries.getLeaderId());
         sender.tell(reply, actor());
 
         // If commitIndex > lastApplied: increment lastApplied, apply
@@ -302,7 +305,7 @@ public class Follower extends AbstractRaftActorBehavior {
         if (appendEntries.getLeaderCommit() > context.getLastApplied()
                 && context.getLastApplied() < lastIndex) {
             if (log.isDebugEnabled()) {
-                log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {},"
+                log.debug("{}: applyLogToStateMachine, appendEntries.getLeaderCommit(): {}, "
                         + "context.getLastApplied(): {}, lastIndex(): {}", logName(),
                     appendEntries.getLeaderCommit(), context.getLastApplied(), lastIndex);
             }
@@ -317,73 +320,80 @@ public class Follower extends AbstractRaftActorBehavior {
         return this;
     }
 
-    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;
+    private boolean isOutOfSync(final AppendEntries appendEntries) {
 
+        final 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.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.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()) {
-
-            // 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.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());
-        } else if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
-                && appendEntries.getReplicatedToAllIndex() != -1
-                && !isLogEntryPresent(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.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.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;
+            // 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.info("{}: The followers log is empty and the senders prevLogIndex is {}", logName(),
+                appendEntries.getPrevLogIndex());
+            return true;
         }
-        return outOfSync;
+
+        if (lastIndex > -1) {
+            if (isLogEntryPresent(appendEntries.getPrevLogIndex())) {
+                final long prevLogTerm = getLogEntryTerm(appendEntries.getPrevLogIndex());
+                if (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.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());
+                    return true;
+                }
+            } else if (appendEntries.getPrevLogIndex() != -1) {
+
+                // The follower's log is out of sync because the Leader's prevLogIndex entry was not found in it's log
+
+                log.info("{}: The log is not empty but the prevLogIndex {} was not found in it - lastIndex: {}, "
+                        + "snapshotIndex: {}", logName(), appendEntries.getPrevLogIndex(), lastIndex,
+                        context.getReplicatedLog().getSnapshotIndex());
+                return true;
+            }
+        }
+
+        if (appendEntries.getPrevLogIndex() == -1 && appendEntries.getPrevLogTerm() == -1
+                && appendEntries.getReplicatedToAllIndex() != -1) {
+            if (!isLogEntryPresent(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.info("{}: Cannot append entries because the replicatedToAllIndex {} does not appear to be in the "
+                        + "in-memory journal", logName(), appendEntries.getReplicatedToAllIndex());
+                return true;
+            }
+
+            final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
+            if (entries != null && entries.size() > 0 && !isLogEntryPresent(entries.get(0).getIndex() - 1)) {
+                log.info("{}: Cannot append entries because the calculated previousIndex {} was not found in the "
+                        + "in-memory journal", logName(), entries.get(0).getIndex() - 1);
+                return true;
+            }
+        }
+
+        return false;
     }
 
     @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);
         }
@@ -419,7 +429,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
@@ -512,7 +522,7 @@ 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);