Fix incorrect spelling of fileBackedStreamFactory
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / Follower.java
index f8524b5174b9c3916e43f73aa9f35e6bbf91bd43..11fa23748e776c11372b74e8e885117db0f06fd1 100644 (file)
@@ -20,11 +20,13 @@ 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;
 import java.util.concurrent.atomic.AtomicBoolean;
 import javax.annotation.Nullable;
+import org.opendaylight.controller.cluster.messaging.MessageAssembler;
 import org.opendaylight.controller.cluster.raft.RaftActorContext;
 import org.opendaylight.controller.cluster.raft.RaftState;
 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
@@ -55,6 +57,8 @@ public class Follower extends AbstractRaftActorBehavior {
 
     private final SyncStatusTracker initialSyncStatusTracker;
 
+    private final MessageAssembler appendEntriesMessageAssembler;
+
     private final Stopwatch lastLeaderMessageTimer = Stopwatch.createStarted();
     private SnapshotTracker snapshotTracker = null;
     private String leaderId;
@@ -73,6 +77,10 @@ public class Follower extends AbstractRaftActorBehavior {
         initialSyncStatusTracker = new SyncStatusTracker(context.getActor(), getId(), context.getConfigParams()
             .getSyncIndexThreshold());
 
+        appendEntriesMessageAssembler = MessageAssembler.builder().logContext(logName())
+                .fileBackedStreamFactory(context.getFileBackedOutputStreamFactory())
+                .assembledMessageCallback((message, sender) -> handleMessage(sender, message)).build();
+
         if (context.getPeerIds().isEmpty() && getLeaderId() == null) {
             actor().tell(TimeoutNow.INSTANCE, actor());
         } else {
@@ -125,7 +133,7 @@ public class Follower extends AbstractRaftActorBehavior {
     @Override
     protected RaftActorBehavior handleAppendEntries(final ActorRef sender, final AppendEntries appendEntries) {
 
-        int numLogEntries = appendEntries.getEntries() != null ? appendEntries.getEntries().size() : 0;
+        int numLogEntries = appendEntries.getEntries().size();
         if (log.isTraceEnabled()) {
             log.trace("{}: handleAppendEntries: {}", logName(), appendEntries);
         } else if (log.isDebugEnabled() && numLogEntries > 0) {
@@ -173,10 +181,8 @@ public class Follower extends AbstractRaftActorBehavior {
             return this;
         }
 
-        if (appendEntries.getEntries() != null && appendEntries.getEntries().size() > 0) {
-
-            log.debug("{}: Number of entries to be appended = {}", logName(),
-                        appendEntries.getEntries().size());
+        if (numLogEntries > 0) {
+            log.debug("{}: Number of entries to be appended = {}", logName(), numLogEntries);
 
             // 3. If an existing entry conflicts with a new one (same index
             // but different terms), delete the existing entry and all that
@@ -185,7 +191,7 @@ public class Follower extends AbstractRaftActorBehavior {
             if (context.getReplicatedLog().size() > 0) {
 
                 // 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++) {
+                for (int i = 0;i < numLogEntries; i++, addEntriesFrom++) {
                     ReplicatedLogEntry matchEntry = appendEntries.getEntries().get(i);
 
                     if (!isLogEntryPresent(matchEntry.getIndex())) {
@@ -245,15 +251,15 @@ public class Follower extends AbstractRaftActorBehavior {
             // purged from the persisted log as well.
             final AtomicBoolean shouldCaptureSnapshot = new AtomicBoolean(false);
             final Procedure<ReplicatedLogEntry> appendAndPersistCallback = logEntry -> {
-                final ReplicatedLogEntry lastEntryToAppend = appendEntries.getEntries().get(
-                        appendEntries.getEntries().size() - 1);
+                final List<ReplicatedLogEntry> entries = appendEntries.getEntries();
+                final ReplicatedLogEntry lastEntryToAppend = entries.get(entries.size() - 1);
                 if (shouldCaptureSnapshot.get() && logEntry == lastEntryToAppend) {
                     context.getSnapshotManager().capture(context.getReplicatedLog().last(), getReplicatedToAllIndex());
                 }
             };
 
             // 4. Append any new entries not already in the log
-            for (int i = addEntriesFrom; i < appendEntries.getEntries().size(); i++) {
+            for (int i = addEntriesFrom; i < numLogEntries; i++) {
                 ReplicatedLogEntry entry = appendEntries.getEntries().get(i);
 
                 log.debug("{}: Append entry to log {}", logName(), entry.getData());
@@ -304,7 +310,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);
             }
@@ -316,60 +322,69 @@ public class Follower extends AbstractRaftActorBehavior {
             super.performSnapshotWithoutCapture(appendEntries.getReplicatedToAllIndex());
         }
 
+        appendEntriesMessageAssembler.checkExpiredAssembledMessageState();
+
         return this;
     }
 
     private boolean isOutOfSync(final 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;
-
+        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.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
@@ -390,6 +405,10 @@ public class Follower extends AbstractRaftActorBehavior {
             return handleElectionTimeout(message);
         }
 
+        if (appendEntriesMessageAssembler.handleMessage(message, actor())) {
+            return this;
+        }
+
         if (!(message instanceof RaftRPC)) {
             // The rest of the processing requires the message to be a RaftRPC
             return null;
@@ -590,6 +609,7 @@ public class Follower extends AbstractRaftActorBehavior {
     public void close() {
         closeSnapshotTracker();
         stopElection();
+        appendEntriesMessageAssembler.close();
     }
 
     @VisibleForTesting