Reset snapshot progress after timeout has been hit
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 15dbd74d01d9b2d3b3aed4b98fdb414b3943c5ae..ec465935039eb99a0c34341e05aa93bd2340e34d 100644 (file)
@@ -5,16 +5,17 @@
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
  * and is available at http://www.eclipse.org/legal/epl-v10.html
  */
-
 package org.opendaylight.controller.cluster.raft.behaviors;
 
+import static java.util.Objects.requireNonNull;
+
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
 import akka.actor.Cancellable;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
 import com.google.common.io.ByteSource;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.io.IOException;
 import java.io.ObjectOutputStream;
 import java.util.Collection;
@@ -27,7 +28,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
+import org.eclipse.jdt.annotation.Nullable;
 import org.opendaylight.controller.cluster.io.SharedFileBackedOutputStream;
 import org.opendaylight.controller.cluster.messaging.MessageSlicer;
 import org.opendaylight.controller.cluster.messaging.SliceOptions;
@@ -101,7 +102,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     private int minReplicationCount;
 
     protected AbstractLeader(final RaftActorContext context, final RaftState state,
-            @Nullable final AbstractLeader initializeFromLeader) {
+            final @Nullable AbstractLeader initializeFromLeader) {
         super(context, state);
 
         appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
@@ -184,7 +185,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    void setSnapshotHolder(@Nullable final SnapshotHolder snapshotHolder) {
+    void setSnapshotHolder(final @Nullable SnapshotHolder snapshotHolder) {
         this.snapshotHolder = Optional.fromNullable(snapshotHolder);
     }
 
@@ -227,9 +228,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         followerLogInformation.markFollowerActive();
         followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion());
         followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion());
+        followerLogInformation.setNeedsLeaderAddress(appendEntriesReply.isNeedsLeaderAddress());
 
         long followerLastLogIndex = appendEntriesReply.getLogLastIndex();
-        long followersLastLogTermInLeadersLog = getLogEntryTerm(followerLastLogIndex);
         boolean updated = false;
         if (appendEntriesReply.getLogLastIndex() > context.getReplicatedLog().lastIndex()) {
             // The follower's log is actually ahead of the leader's log. Normally this doesn't happen
@@ -246,9 +247,10 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             // However in this case the log terms won't match and the logs will conflict - this is handled
             // elsewhere.
             log.info("{}: handleAppendEntriesReply: follower {} lastIndex {} is ahead of our lastIndex {} "
-                    + "(snapshotIndex {}) - forcing install snaphot", logName(), followerLogInformation.getId(),
-                    appendEntriesReply.getLogLastIndex(), context.getReplicatedLog().lastIndex(),
-                    context.getReplicatedLog().getSnapshotIndex());
+                    + "(snapshotIndex {}, snapshotTerm {}) - forcing install snaphot", logName(),
+                    followerLogInformation.getId(), appendEntriesReply.getLogLastIndex(),
+                    context.getReplicatedLog().lastIndex(), context.getReplicatedLog().getSnapshotIndex(),
+                    context.getReplicatedLog().getSnapshotTerm());
 
             followerLogInformation.setMatchIndex(-1);
             followerLogInformation.setNextIndex(-1);
@@ -257,6 +259,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
             updated = true;
         } else if (appendEntriesReply.isSuccess()) {
+            long followersLastLogTermInLeadersLog = getLogEntryTerm(followerLastLogIndex);
             if (followerLastLogIndex >= 0 && followersLastLogTermInLeadersLog >= 0
                     && followersLastLogTermInLeadersLog != appendEntriesReply.getLogLastTerm()) {
                 // The follower's last entry is present in the leader's journal but the terms don't match so the
@@ -278,9 +281,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
             }
         } else {
-            log.info("{}: handleAppendEntriesReply - received unsuccessful reply: {}, leader snapshotIndex: {}",
-                    logName(), appendEntriesReply, context.getReplicatedLog().getSnapshotIndex());
+            log.info("{}: handleAppendEntriesReply - received unsuccessful reply: {}, leader snapshotIndex: {}, "
+                    + "snapshotTerm: {}, replicatedToAllIndex: {}", logName(), appendEntriesReply,
+                    context.getReplicatedLog().getSnapshotIndex(), context.getReplicatedLog().getSnapshotTerm(),
+                    getReplicatedToAllIndex());
 
+            long followersLastLogTermInLeadersLogOrSnapshot = getLogEntryOrSnapshotTerm(followerLastLogIndex);
             if (appendEntriesReply.isForceInstallSnapshot()) {
                 // Reset the followers match and next index. This is to signal that this follower has nothing
                 // in common with this Leader and so would require a snapshot to be installed
@@ -289,12 +295,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
                 // Force initiate a snapshot capture
                 initiateCaptureSnapshot(followerId);
-            } else if (followerLastLogIndex < 0 || followersLastLogTermInLeadersLog >= 0
-                    && followersLastLogTermInLeadersLog == appendEntriesReply.getLogLastTerm()) {
-                // The follower's log is empty or the last entry is present in the leader's journal
-                // and the terms match so the follower is just behind the leader's journal from
-                // the last snapshot, if any. We'll catch up the follower quickly by starting at the
-                // follower's last log index.
+            } else if (followerLastLogIndex < 0 || followersLastLogTermInLeadersLogOrSnapshot >= 0
+                    && followersLastLogTermInLeadersLogOrSnapshot == appendEntriesReply.getLogLastTerm()) {
+                // The follower's log is empty or the follower's last entry is present in the leader's journal or
+                // snapshot and the terms match so the follower is just behind the leader's journal from the last
+                // snapshot, if any. We'll catch up the follower quickly by starting at the follower's last log index.
 
                 updated = updateFollowerLogInformation(followerLogInformation, appendEntriesReply);
 
@@ -302,15 +307,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
                     followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
             } else {
-                // The follower's log conflicts with leader's log so decrement follower's next index by 1
+                // The follower's log conflicts with leader's log so decrement follower's next index
                 // in an attempt to find where the logs match.
-
-                if (followerLogInformation.decrNextIndex()) {
+                if (followerLogInformation.decrNextIndex(appendEntriesReply.getLogLastIndex())) {
                     updated = true;
 
                     log.info("{}: follower {} last log term {} conflicts with the leader's {} - dec next index to {}",
                             logName(), followerId, appendEntriesReply.getLogLastTerm(),
-                            followersLastLogTermInLeadersLog, followerLogInformation.getNextIndex());
+                            followersLastLogTermInLeadersLogOrSnapshot, followerLogInformation.getNextIndex());
                 }
             }
         }
@@ -452,7 +456,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
     @Override
     public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
-        Preconditions.checkNotNull(sender, "sender should not be null");
+        requireNonNull(sender, "sender should not be null");
 
         if (appendEntriesMessageSlicer.handleMessage(message)) {
             return this;
@@ -508,6 +512,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return this;
     }
 
+    @SuppressFBWarnings(value = "NP_NULL_PARAM_DEREF_ALL_TARGETS_DANGEROUS",
+            justification = "JDT nullness with SpotBugs at setSnapshotHolder(null)")
     private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
 
@@ -527,6 +533,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
             return;
         }
 
+        installSnapshotState.resetChunkTimer();
         followerLogInformation.markFollowerActive();
 
         if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) {
@@ -658,9 +665,18 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
             LeaderInstallSnapshotState installSnapshotState = followerLogInformation.getInstallSnapshotState();
             if (installSnapshotState != null) {
+
                 // if install snapshot is in process , then sent next chunk if possible
-                if (isFollowerActive && installSnapshotState.canSendNextChunk()) {
-                    sendSnapshotChunk(followerActor, followerLogInformation);
+                if (isFollowerActive) {
+                    // 30 seconds with default settings, can be modified via heartbeat or election timeout factor
+                    FiniteDuration snapshotReplyTimeout = context.getConfigParams().getHeartBeatInterval()
+                            .$times(context.getConfigParams().getElectionTimeoutFactor() * 3);
+
+                    if (installSnapshotState.isChunkTimedOut(snapshotReplyTimeout)) {
+                        sendAppendEntries = !resendSnapshotChunk(followerActor, followerLogInformation);
+                    } else if (installSnapshotState.canSendNextChunk()) {
+                        sendSnapshotChunk(followerActor, followerLogInformation);
+                    }
                 } else if (sendHeartbeat) {
                     // we send a heartbeat even if we have not received a reply for the last chunk
                     sendAppendEntries = true;
@@ -813,7 +829,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
             getLogEntryIndex(followerNextIndex - 1),
             getLogEntryTerm(followerNextIndex - 1), entries,
-            leaderCommitIndex, super.getReplicatedToAllIndex(), context.getPayloadVersion());
+            leaderCommitIndex, super.getReplicatedToAllIndex(), context.getPayloadVersion(),
+            followerLogInformation.getRaftVersion(), followerLogInformation.needsLeaderAddress(getId()));
 
         if (!entries.isEmpty() || log.isTraceEnabled()) {
             log.debug("{}: Sending AppendEntries to follower {}: {}", logName(), followerLogInformation.getId(),
@@ -924,18 +941,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     serverConfig = Optional.fromNullable(context.getPeerServerInfo(true));
                 }
 
-                followerActor.tell(
-                    new InstallSnapshot(currentTerm(), context.getId(),
-                        snapshotHolder.get().getLastIncludedIndex(),
-                        snapshotHolder.get().getLastIncludedTerm(),
-                        nextSnapshotChunk,
-                        nextChunkIndex,
-                        installSnapshotState.getTotalChunks(),
-                        Optional.of(installSnapshotState.getLastChunkHashCode()),
-                        serverConfig
-                    ).toSerializable(followerLogInfo.getRaftVersion()),
-                    actor()
-                );
+                sendSnapshotChunk(followerActor, followerLogInfo, nextSnapshotChunk, nextChunkIndex, serverConfig);
 
             } catch (IOException e) {
                 throw new RuntimeException(e);
@@ -946,6 +952,46 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
+    private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo,
+                                   final byte[] snapshotChunk, final int chunkIndex,
+                                   final Optional<ServerConfigurationPayload> serverConfig) {
+        LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
+
+        installSnapshotState.startChunkTimer();
+        followerActor.tell(
+                new InstallSnapshot(currentTerm(), context.getId(),
+                        snapshotHolder.get().getLastIncludedIndex(),
+                        snapshotHolder.get().getLastIncludedTerm(),
+                        snapshotChunk,
+                        chunkIndex,
+                        installSnapshotState.getTotalChunks(),
+                        Optional.of(installSnapshotState.getLastChunkHashCode()),
+                        serverConfig
+                ).toSerializable(followerLogInfo.getRaftVersion()),
+                actor()
+        );
+    }
+
+    private boolean resendSnapshotChunk(final ActorSelection followerActor,
+                                        final FollowerLogInformation followerLogInfo) {
+        if (!snapshotHolder.isPresent()) {
+            // Seems like we should never hit this case, but just in case we do, reset the snapshot progress so that it
+            // can restart from the next AppendEntries.
+            log.warn("{}: Attempting to resend snapshot with no snapshot holder present.", logName());
+            followerLogInfo.clearLeaderInstallSnapshotState();
+            return false;
+        }
+
+        LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
+        // we are resending, timer needs to be reset
+        installSnapshotState.resetChunkTimer();
+        installSnapshotState.markSendStatus(false);
+
+        sendSnapshotChunk(followerActor, followerLogInfo);
+
+        return true;
+    }
+
     private void sendHeartBeat() {
         if (!followerToLog.isEmpty()) {
             log.trace("{}: Sending heartbeat", logName());