Fix intermittent testOwnerChangesOnPeerAvailabilityChanges failure
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractLeader.java
index 097f0ec677ea52355d99a1256c3a219ccf522869..a40e75f4c3b2e5629cf2904891a61c7819ee12c9 100644 (file)
@@ -14,9 +14,9 @@ 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.base.Throwables;
 import com.google.common.io.ByteSource;
 import java.io.IOException;
+import java.io.ObjectOutputStream;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -26,7 +26,11 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
+import java.util.concurrent.TimeUnit;
 import javax.annotation.Nullable;
+import org.opendaylight.controller.cluster.io.SharedFileBackedOutputStream;
+import org.opendaylight.controller.cluster.messaging.MessageSlicer;
+import org.opendaylight.controller.cluster.messaging.SliceOptions;
 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
 import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl;
 import org.opendaylight.controller.cluster.raft.FollowerLogInformation;
@@ -85,14 +89,27 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      */
     private final Queue<ClientRequestTracker> trackers = new LinkedList<>();
 
+    /**
+     * Map of serialized AppendEntries output streams keyed by log index. This is used in conjunction with the
+     * appendEntriesMessageSlicer for slicing single ReplicatedLogEntry payloads that exceed the message size threshold.
+     * This Map allows the SharedFileBackedOutputStreams to be reused for multiple followers.
+     */
+    private final Map<Long, SharedFileBackedOutputStream> sharedSerializedAppendEntriesStreams = new HashMap<>();
+    private final MessageSlicer appendEntriesMessageSlicer;
+
     private Cancellable heartbeatSchedule = null;
     private Optional<SnapshotHolder> snapshotHolder = Optional.absent();
     private int minReplicationCount;
 
-    protected AbstractLeader(RaftActorContext context, RaftState state,
-            @Nullable AbstractLeader initializeFromLeader) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state,
+            @Nullable final AbstractLeader initializeFromLeader) {
         super(context, state);
 
+        appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName())
+            .messageSliceSize(context.getConfigParams().getSnapshotChunkSize())
+            .expireStateAfterInactivity(context.getConfigParams().getElectionTimeOutInterval().toMillis() * 3,
+                    TimeUnit.MILLISECONDS).build();
+
         if (initializeFromLeader != null) {
             followerToLog.putAll(initializeFromLeader.followerToLog);
             snapshotHolder = initializeFromLeader.snapshotHolder;
@@ -118,7 +135,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval());
     }
 
-    protected AbstractLeader(RaftActorContext context, RaftState state) {
+    protected AbstractLeader(final RaftActorContext context, final RaftState state) {
         this(context, state, null);
     }
 
@@ -131,7 +148,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return followerToLog.keySet();
     }
 
-    public void addFollower(String followerId) {
+    public void addFollower(final String followerId) {
         FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(
                 context.getPeerInfo(followerId), -1, context);
         followerToLog.put(followerId, followerLogInformation);
@@ -141,7 +158,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    public void removeFollower(String followerId) {
+    public void removeFollower(final String followerId) {
         followerToLog.remove(followerId);
     }
 
@@ -168,7 +185,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    void setSnapshot(@Nullable SnapshotHolder snapshotHolder) {
+    void setSnapshot(@Nullable final SnapshotHolder snapshotHolder) {
         this.snapshotHolder = Optional.fromNullable(snapshotHolder);
     }
 
@@ -178,8 +195,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntries(ActorRef sender,
-        AppendEntries appendEntries) {
+    protected RaftActorBehavior handleAppendEntries(final ActorRef sender,
+        final AppendEntries appendEntries) {
 
         log.debug("{}: handleAppendEntries: {}", logName(), appendEntries);
 
@@ -187,7 +204,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleAppendEntriesReply(ActorRef sender, AppendEntriesReply appendEntriesReply) {
+    protected RaftActorBehavior handleAppendEntriesReply(final ActorRef sender,
+            final AppendEntriesReply appendEntriesReply) {
         log.trace("{}: handleAppendEntriesReply: {}", logName(), appendEntriesReply);
 
         // Update the FollowerLogInformation
@@ -283,8 +301,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
 
                 log.info("{}: follower {} appears to be behind the leader from the last snapshot - "
                     + "updated: matchIndex: {}, nextIndex: {}", logName(), followerId,
-                    appendEntriesReply.getLogLastTerm(), followerLogInformation.getMatchIndex(),
-                    followerLogInformation.getNextIndex());
+                    followerLogInformation.getMatchIndex(), followerLogInformation.getNextIndex());
             } else {
                 // The follower's log conflicts with leader's log so decrement follower's next index by 1
                 // in an attempt to find where the logs match.
@@ -385,8 +402,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private boolean updateFollowerLogInformation(FollowerLogInformation followerLogInformation,
-            AppendEntriesReply appendEntriesReply) {
+    private boolean updateFollowerLogInformation(final FollowerLogInformation followerLogInformation,
+            final AppendEntriesReply appendEntriesReply) {
         boolean updated = followerLogInformation.setMatchIndex(appendEntriesReply.getLogLastIndex());
         updated = followerLogInformation.setNextIndex(appendEntriesReply.getLogLastIndex() + 1) || updated;
 
@@ -413,7 +430,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
+    protected ClientRequestTracker removeClientRequestTracker(final long logIndex) {
         final Iterator<ClientRequestTracker> it = trackers.iterator();
         while (it.hasNext()) {
             final ClientRequestTracker t = it.next();
@@ -427,17 +444,21 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @Override
-    protected RaftActorBehavior handleRequestVoteReply(ActorRef sender,
-        RequestVoteReply requestVoteReply) {
+    protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender,
+        final RequestVoteReply requestVoteReply) {
         return this;
     }
 
     protected void beforeSendHeartbeat(){}
 
     @Override
-    public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
+    public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
         Preconditions.checkNotNull(sender, "sender should not be null");
 
+        if (appendEntriesMessageSlicer.handleMessage(message)) {
+            return this;
+        }
+
         if (message instanceof RaftRPC) {
             RaftRPC rpc = (RaftRPC) message;
             // If RPC request or response contains term T > currentTerm:
@@ -487,7 +508,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return this;
     }
 
-    private void handleInstallSnapshotReply(InstallSnapshotReply reply) {
+    private void handleInstallSnapshotReply(final InstallSnapshotReply reply) {
         log.debug("{}: handleInstallSnapshotReply: {}", logName(), reply);
 
         String followerId = reply.getFollowerId();
@@ -581,7 +602,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return false;
     }
 
-    private void replicate(Replicate replicate) {
+    private void replicate(final Replicate replicate) {
         long logIndex = replicate.getReplicatedLogEntry().getIndex();
 
         log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(),
@@ -608,7 +629,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) {
+    protected void sendAppendEntries(final long timeSinceLastActivityInterval, final boolean isHeartbeat) {
         // Send an AppendEntries to all followers
         for (Entry<String, FollowerLogInformation> e : followerToLog.entrySet()) {
             final String followerId = e.getKey();
@@ -625,8 +646,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * This method checks if any update needs to be sent to the given follower. This includes append log entries,
      * sending next snapshot chunk, and initiating a snapshot.
      */
-    private void sendUpdatesToFollower(String followerId, FollowerLogInformation followerLogInformation,
-                                       boolean sendHeartbeat, boolean isHeartbeat) {
+    private void sendUpdatesToFollower(final String followerId, final FollowerLogInformation followerLogInformation,
+                                       final boolean sendHeartbeat, final boolean isHeartbeat) {
 
         ActorSelection followerActor = context.getPeerActorSelection(followerId);
         if (followerActor != null) {
@@ -644,6 +665,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     // we send a heartbeat even if we have not received a reply for the last chunk
                     sendAppendEntries = true;
                 }
+            } else if (followerLogInformation.isLogEntrySlicingInProgress()) {
+                sendAppendEntries = sendHeartbeat;
             } else {
                 long leaderLastIndex = context.getReplicatedLog().lastIndex();
                 long leaderSnapShotIndex = context.getReplicatedLog().getSnapshotIndex();
@@ -660,11 +683,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                             followerNextIndex, followerId);
 
                     if (followerLogInformation.okToReplicate()) {
-                        // Try to send all the entries in the journal but not exceeding the max data size
-                        // for a single AppendEntries message.
-                        int maxEntries = (int) context.getReplicatedLog().size();
-                        entries = context.getReplicatedLog().getFrom(followerNextIndex, maxEntries,
-                                context.getConfigParams().getSnapshotChunkSize());
+                        entries = getEntriesToSend(followerLogInformation, followerActor);
                         sendAppendEntries = true;
                     }
                 } else if (isFollowerActive && followerNextIndex >= 0
@@ -705,6 +724,76 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
+    private List<ReplicatedLogEntry> getEntriesToSend(FollowerLogInformation followerLogInfo,
+            ActorSelection followerActor) {
+        // Try to get all the entries in the journal but not exceeding the max data size for a single AppendEntries
+        // message.
+        int maxEntries = (int) context.getReplicatedLog().size();
+        final int maxDataSize = context.getConfigParams().getSnapshotChunkSize();
+        final long followerNextIndex = followerLogInfo.getNextIndex();
+        List<ReplicatedLogEntry> entries = context.getReplicatedLog().getFrom(followerNextIndex,
+                maxEntries, maxDataSize);
+
+        // If the first entry's size exceeds the max data size threshold, it will be returned from the call above. If
+        // that is the case, then we need to slice it into smaller chunks.
+        if (!(entries.size() == 1 && entries.get(0).getData().size() > maxDataSize)) {
+            // Don't need to slice.
+            return entries;
+        }
+
+        log.debug("{}: Log entry size {} exceeds max payload size {}", logName(), entries.get(0).getData().size(),
+                maxDataSize);
+
+        // If an AppendEntries has already been serialized for the log index then reuse the
+        // SharedFileBackedOutputStream.
+        final Long logIndex = entries.get(0).getIndex();
+        SharedFileBackedOutputStream fileBackedStream = sharedSerializedAppendEntriesStreams.get(logIndex);
+        if (fileBackedStream == null) {
+            fileBackedStream = context.getFileBackedOutputStreamFactory().newSharedInstance();
+
+            final AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
+                    getLogEntryIndex(followerNextIndex - 1), getLogEntryTerm(followerNextIndex - 1), entries,
+                    context.getCommitIndex(), getReplicatedToAllIndex(), context.getPayloadVersion());
+
+            log.debug("{}: Serializing {} for slicing for follower {}", logName(), appendEntries,
+                    followerLogInfo.getId());
+
+            try (ObjectOutputStream out = new ObjectOutputStream(fileBackedStream)) {
+                out.writeObject(appendEntries);
+            } catch (IOException e) {
+                log.error("{}: Error serializing {}", logName(), appendEntries, e);
+                fileBackedStream.cleanup();
+                return Collections.emptyList();
+            }
+
+            sharedSerializedAppendEntriesStreams.put(logIndex, fileBackedStream);
+
+            fileBackedStream.setOnCleanupCallback(index -> {
+                log.debug("{}: On SharedFileBackedOutputStream cleanup for index {}", logName(), index);
+                sharedSerializedAppendEntriesStreams.remove(index);
+            }, logIndex);
+        } else {
+            log.debug("{}: Reusing SharedFileBackedOutputStream for follower {}", logName(), followerLogInfo.getId());
+            fileBackedStream.incrementUsageCount();
+        }
+
+        log.debug("{}: Slicing stream for index {}, follower {}", logName(), logIndex, followerLogInfo.getId());
+
+        // Record that slicing is in progress for the follower.
+        followerLogInfo.setSlicedLogEntryIndex(logIndex);
+
+        final FollowerIdentifier identifier = new FollowerIdentifier(followerLogInfo.getId());
+        appendEntriesMessageSlicer.slice(SliceOptions.builder().identifier(identifier)
+                .fileBackedOutputStream(fileBackedStream).sendTo(followerActor).replyTo(actor())
+                .onFailureCallback(failure -> {
+                    log.error("{}: Error slicing AppendEntries for follower {}", logName(),
+                            followerLogInfo.getId(), failure);
+                    followerLogInfo.setSlicedLogEntryIndex(FollowerLogInformation.NO_INDEX);
+                }).build());
+
+        return Collections.emptyList();
+    }
+
     private void sendAppendEntriesToFollower(ActorSelection followerActor, List<ReplicatedLogEntry> entries,
             FollowerLogInformation followerLogInformation) {
         // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from
@@ -714,9 +803,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         //       empty AppendEntries as a heart beat to prevent election.
         //     - if we're in the process of installing a snapshot. In this case we don't send any new entries but still
         //       need to send AppendEntries to prevent election.
+        //     - if we're in the process of slicing an AppendEntries with a large log entry payload. In this case we
+        //       need to send an empty AppendEntries to prevent election.
         boolean isInstallingSnaphot = followerLogInformation.getInstallSnapshotState() != null;
-        long leaderCommitIndex = isInstallingSnaphot || !followerLogInformation.isFollowerActive() ? -1 :
-            context.getCommitIndex();
+        long leaderCommitIndex = isInstallingSnaphot || followerLogInformation.isLogEntrySlicingInProgress()
+                || !followerLogInformation.isFollowerActive() ? -1 : context.getCommitIndex();
 
         long followerNextIndex = followerLogInformation.getNextIndex();
         AppendEntries appendEntries = new AppendEntries(currentTerm(), context.getId(),
@@ -751,7 +842,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      * @param followerId the id of the follower.
      * @return true if capture was initiated, false otherwise.
      */
-    public boolean initiateCaptureSnapshot(String followerId) {
+    public boolean initiateCaptureSnapshot(final String followerId) {
         FollowerLogInformation followerLogInfo = followerToLog.get(followerId);
         if (snapshotHolder.isPresent()) {
             // If a snapshot is present in the memory, most likely another install is in progress no need to capture
@@ -773,7 +864,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         return captureInitiated;
     }
 
-    private boolean canInstallSnapshot(long nextIndex) {
+    private boolean canInstallSnapshot(final long nextIndex) {
         // If the follower's nextIndex is -1 then we might as well send it a snapshot
         // Otherwise send it a snapshot only if the nextIndex is not present in the log but is present
         // in the snapshot
@@ -805,7 +896,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
      *  Sends a snapshot chunk to a given follower
      *  InstallSnapshot should qualify as a heartbeat too.
      */
-    private void sendSnapshotChunk(ActorSelection followerActor, FollowerLogInformation followerLogInfo) {
+    private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo) {
         if (snapshotHolder.isPresent()) {
             LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState();
             if (installSnapshotState == null) {
@@ -846,11 +937,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
                     actor()
                 );
 
-                log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
-                        installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
             } catch (IOException e) {
-                throw Throwables.propagate(e);
+                throw new RuntimeException(e);
             }
+
+            log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(),
+                installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks());
         }
     }
 
@@ -858,6 +950,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         if (!followerToLog.isEmpty()) {
             log.trace("{}: Sending heartbeat", logName());
             sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true);
+
+            appendEntriesMessageSlicer.checkExpiredSlicedMessageState();
         }
     }
 
@@ -867,7 +961,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         }
     }
 
-    private void scheduleHeartBeat(FiniteDuration interval) {
+    private void scheduleHeartBeat(final FiniteDuration interval) {
         if (followerToLog.isEmpty()) {
             // Optimization - do not bother scheduling a heartbeat as there are
             // no followers
@@ -889,6 +983,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     @Override
     public void close() {
         stopHeartBeat();
+        appendEntriesMessageSlicer.close();
     }
 
     @Override
@@ -933,7 +1028,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
     }
 
     @VisibleForTesting
-    public FollowerLogInformation getFollower(String followerId) {
+    public FollowerLogInformation getFollower(final String followerId) {
         return followerToLog.get(followerId);
     }
 
@@ -947,7 +1042,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior {
         private final long lastIncludedIndex;
         private final ByteSource snapshotBytes;
 
-        SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) {
+        SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) {
             this.lastIncludedTerm = snapshot.getLastAppliedTerm();
             this.lastIncludedIndex = snapshot.getLastAppliedIndex();
             this.snapshotBytes = snapshotBytes;