X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FAbstractLeader.java;h=0188a6df1ac387e603962d838def314a7dab1c8d;hp=9fc7b3393cbe1f86404e2eb9a08a9ca6f6e21bc1;hb=86e8e4a06b682aa772c834a2cef56d0596540e1b;hpb=8d90cf04be86f872f7eeb892d37517d5ee087157 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java index 9fc7b3393c..0188a6df1a 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/behaviors/AbstractLeader.java @@ -5,17 +5,16 @@ * 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.base.Throwables; 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; @@ -26,29 +25,33 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Optional; +import java.util.OptionalInt; 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; import org.opendaylight.controller.cluster.raft.ClientRequestTracker; import org.opendaylight.controller.cluster.raft.ClientRequestTrackerImpl; import org.opendaylight.controller.cluster.raft.FollowerLogInformation; -import org.opendaylight.controller.cluster.raft.FollowerLogInformationImpl; import org.opendaylight.controller.cluster.raft.PeerInfo; 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.VotingState; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.CheckConsensusReached; import org.opendaylight.controller.cluster.raft.base.messages.Replicate; import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat; import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot; import org.opendaylight.controller.cluster.raft.messages.AppendEntries; import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +import org.opendaylight.controller.cluster.raft.messages.IdentifiablePayload; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshot; import org.opendaylight.controller.cluster.raft.messages.InstallSnapshotReply; +import org.opendaylight.controller.cluster.raft.messages.Payload; import org.opendaylight.controller.cluster.raft.messages.RaftRPC; import org.opendaylight.controller.cluster.raft.messages.RequestVote; import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply; @@ -58,7 +61,7 @@ import org.opendaylight.controller.cluster.raft.persisted.Snapshot; import scala.concurrent.duration.FiniteDuration; /** - * The behavior of a RaftActor when it is in the Leader state + * The behavior of a RaftActor when it is in the Leader state. * *

* Leaders: @@ -99,11 +102,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private final MessageSlicer appendEntriesMessageSlicer; private Cancellable heartbeatSchedule = null; - private Optional snapshotHolder = Optional.absent(); + private Optional snapshotHolder = Optional.empty(); private int minReplicationCount; - protected AbstractLeader(RaftActorContext context, RaftState state, - @Nullable AbstractLeader initializeFromLeader) { + protected AbstractLeader(final RaftActorContext context, final RaftState state, + final @Nullable AbstractLeader initializeFromLeader) { super(context, state); appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName()) @@ -117,7 +120,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { trackers.addAll(initializeFromLeader.trackers); } else { for (PeerInfo peerInfo: context.getPeers()) { - FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl(peerInfo, -1, context); + FollowerLogInformation followerLogInformation = new FollowerLogInformation(peerInfo, context); followerToLog.put(peerInfo.getId(), followerLogInformation); } } @@ -136,7 +139,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); } @@ -149,9 +152,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return followerToLog.keySet(); } - public void addFollower(String followerId) { - FollowerLogInformation followerLogInformation = new FollowerLogInformationImpl( - context.getPeerInfo(followerId), -1, context); + public void addFollower(final String followerId) { + FollowerLogInformation followerLogInformation = new FollowerLogInformation(context.getPeerInfo(followerId), + context); followerToLog.put(followerId, followerLogInformation); if (heartbeatSchedule == null) { @@ -159,11 +162,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - public void removeFollower(String followerId) { + public void removeFollower(final String followerId) { followerToLog.remove(followerId); } - public void updateMinReplicaCount() { + public final void updateMinReplicaCount() { int numVoting = 0; for (PeerInfo peer: context.getPeers()) { if (peer.isVoting()) { @@ -186,8 +189,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @VisibleForTesting - void setSnapshot(@Nullable SnapshotHolder snapshotHolder) { - this.snapshotHolder = Optional.fromNullable(snapshotHolder); + void setSnapshotHolder(final @Nullable SnapshotHolder snapshotHolder) { + this.snapshotHolder = Optional.ofNullable(snapshotHolder); } @VisibleForTesting @@ -196,8 +199,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); @@ -205,7 +208,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 @@ -217,20 +221,20 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } - if (followerLogInformation.timeSinceLastActivity() - > context.getConfigParams().getElectionTimeOutInterval().toMillis()) { + final long lastActivityNanos = followerLogInformation.nanosSinceLastActivity(); + if (lastActivityNanos > context.getConfigParams().getElectionTimeOutInterval().toNanos()) { log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " + "appendEntriesReply : {}, timeSinceLastActivity : {}, lastApplied : {}, commitIndex : {}", - logName(), appendEntriesReply, followerLogInformation.timeSinceLastActivity(), + logName(), appendEntriesReply, TimeUnit.NANOSECONDS.toMillis(lastActivityNanos), context.getLastApplied(), context.getCommitIndex()); } 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 @@ -247,9 +251,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); @@ -258,6 +263,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 @@ -279,9 +285,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 @@ -290,29 +299,26 @@ 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); 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 + // 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()); } } } @@ -403,8 +409,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; @@ -430,8 +436,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { super.performSnapshotWithoutCapture(minReplicatedToAllIndex); } - @Override - protected ClientRequestTracker removeClientRequestTracker(long logIndex) { + /** + * Removes and returns the ClientRequestTracker for the specified log index. + * @param logIndex the log index + * @return the ClientRequestTracker or null if none available + */ + private ClientRequestTracker removeClientRequestTracker(final long logIndex) { final Iterator it = trackers.iterator(); while (it.hasNext()) { final ClientRequestTracker t = it.next(); @@ -445,27 +455,49 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @Override - protected RaftActorBehavior handleRequestVoteReply(ActorRef sender, - RequestVoteReply requestVoteReply) { + final ApplyState getApplyStateFor(final ReplicatedLogEntry entry) { + // first check whether a ClientRequestTracker exists for this entry. + // If it does that means the leader wasn't dropped before the transaction applied. + // That means that this transaction can be safely applied as a local transaction since we + // have the ClientRequestTracker. + final ClientRequestTracker tracker = removeClientRequestTracker(entry.getIndex()); + if (tracker != null) { + return new ApplyState(tracker.getClientActor(), tracker.getIdentifier(), entry); + } + + // Tracker is missing, this means that we switched behaviours between replicate and applystate + // and became the leader again,. We still want to apply this as a local modification because + // we have resumed leadership with that log entry having been committed. + final Payload payload = entry.getData(); + if (payload instanceof IdentifiablePayload) { + return new ApplyState(null, ((IdentifiablePayload) payload).getIdentifier(), entry); + } + + return new ApplyState(null, null, entry); + } + + @Override + protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, final RequestVoteReply requestVoteReply) { return this; } - protected void beforeSendHeartbeat(){} + protected void beforeSendHeartbeat() { + // No-op + } @Override - public RaftActorBehavior handleMessage(ActorRef sender, Object message) { - Preconditions.checkNotNull(sender, "sender should not be null"); + public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) { + requireNonNull(sender, "sender should not be null"); if (appendEntriesMessageSlicer.handleMessage(message)) { return this; } - if (message instanceof RaftRPC) { - RaftRPC rpc = (RaftRPC) message; + if (message instanceof RaftRPC rpc) { // If RPC request or response contains term T > currentTerm: // set currentTerm = T, convert to follower (§5.1) // This applies to all RPC messages and responses - if (rpc.getTerm() > context.getTermInformation().getCurrentTerm()) { + if (rpc.getTerm() > context.getTermInformation().getCurrentTerm() && shouldUpdateTerm(rpc)) { log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower", logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); @@ -479,9 +511,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // start a new election due to lack of responses. This case would only occur if there isn't a majority // of other nodes available that can elect the requesting candidate. Since we're transferring // leadership, we should make every effort to get the requesting node elected. - if (message instanceof RequestVote && context.getRaftActorLeadershipTransferCohort() != null) { + if (rpc instanceof RequestVote && context.getRaftActorLeadershipTransferCohort() != null) { log.debug("{}: Leadership transfer in progress - processing RequestVote", logName()); - super.handleMessage(sender, message); + super.handleMessage(sender, rpc); } return internalSwitchBehavior(RaftState.Follower); @@ -492,9 +524,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { beforeSendHeartbeat(); sendHeartBeat(); scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval()); - } else if (message instanceof SendInstallSnapshot) { - SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message; - setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes())); + } else if (message instanceof SendInstallSnapshot sendInstallSnapshot) { + setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), + sendInstallSnapshot.getSnapshotBytes())); sendInstallSnapshot(); } else if (message instanceof Replicate) { replicate((Replicate) message); @@ -509,7 +541,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } - private void handleInstallSnapshotReply(InstallSnapshotReply reply) { + @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); String followerId = reply.getFollowerId(); @@ -528,6 +562,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return; } + installSnapshotState.resetChunkTimer(); followerLogInformation.markFollowerActive(); if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) { @@ -548,7 +583,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { if (!anyFollowersInstallingSnapshot()) { // once there are no pending followers receiving snapshots // we can remove snapshot from the memory - setSnapshot(null); + setSnapshotHolder(null); } wasLastChunk = true; @@ -603,7 +638,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(), @@ -630,14 +665,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - protected void sendAppendEntries(long timeSinceLastActivityInterval, boolean isHeartbeat) { + protected void sendAppendEntries(final long timeSinceLastActivityIntervalNanos, final boolean isHeartbeat) { // Send an AppendEntries to all followers for (Entry e : followerToLog.entrySet()) { final String followerId = e.getKey(); final FollowerLogInformation followerLogInformation = e.getValue(); // This checks helps not to send a repeat message to the follower if (!followerLogInformation.isFollowerActive() - || followerLogInformation.timeSinceLastActivity() >= timeSinceLastActivityInterval) { + || followerLogInformation.nanosSinceLastActivity() >= timeSinceLastActivityIntervalNanos) { sendUpdatesToFollower(followerId, followerLogInformation, true, isHeartbeat); } } @@ -647,8 +682,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) { @@ -659,10 +694,19 @@ 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); - } else if (sendHeartbeat) { + 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 || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) { // we send a heartbeat even if we have not received a reply for the last chunk sendAppendEntries = true; } @@ -683,7 +727,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { log.debug("{}: sendAppendEntries: {} is present for follower {}", logName(), followerNextIndex, followerId); - if (followerLogInformation.okToReplicate()) { + if (followerLogInformation.okToReplicate(context.getCommitIndex())) { entries = getEntriesToSend(followerLogInformation, followerActor); sendAppendEntries = true; } @@ -711,7 +755,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { context.getReplicatedLog().size()); } - } else if (sendHeartbeat) { + } else if (sendHeartbeat || followerLogInformation.hasStaleCommitIndex(context.getCommitIndex())) { // we send an AppendEntries, even if the follower is inactive // in-order to update the followers timestamp, in case it becomes active again sendAppendEntries = true; @@ -725,8 +769,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - private List getEntriesToSend(FollowerLogInformation followerLogInfo, - ActorSelection followerActor) { + private List getEntriesToSend(final FollowerLogInformation followerLogInfo, + final 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(); @@ -737,7 +781,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // 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)) { + if (entries.size() != 1 || entries.get(0).getData().serializedSize() <= maxDataSize) { // Don't need to slice. return entries; } @@ -795,8 +839,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return Collections.emptyList(); } - private void sendAppendEntriesToFollower(ActorSelection followerActor, List entries, - FollowerLogInformation followerLogInformation) { + private void sendAppendEntriesToFollower(final ActorSelection followerActor, final List entries, + final FollowerLogInformation followerLogInformation) { // In certain cases outlined below we don't want to send the actual commit index to prevent the follower from // possibly committing and applying conflicting entries (those with same index, different term) from a prior // term that weren't replicated to a majority, which would be a violation of raft. @@ -814,13 +858,15 @@ 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(), appendEntries); } + followerLogInformation.setSentCommitIndex(leaderCommitIndex); followerActor.tell(appendEntries, actor()); } @@ -843,7 +889,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 @@ -856,7 +902,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } boolean captureInitiated = context.getSnapshotManager().captureToInstall(context.getReplicatedLog().last(), - this.getReplicatedToAllIndex(), followerId); + getReplicatedToAllIndex(), followerId); if (captureInitiated) { followerLogInfo.setLeaderInstallSnapshotState(new LeaderInstallSnapshotState( context.getConfigParams().getSnapshotChunkSize(), logName())); @@ -865,7 +911,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 @@ -897,7 +943,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) { @@ -920,36 +966,69 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { nextSnapshotChunk.length); int nextChunkIndex = installSnapshotState.incrementChunkIndex(); - Optional serverConfig = Optional.absent(); + Optional serverConfig = Optional.empty(); if (installSnapshotState.isLastChunk(nextChunkIndex)) { - serverConfig = Optional.fromNullable(context.getPeerServerInfo(true)); + serverConfig = Optional.ofNullable(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); log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(), installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks()); + } catch (IOException e) { - throw Throwables.propagate(e); + log.warn("{}: Unable to send chunk: {}/{}. Reseting snapshot progress. Snapshot state: {}", logName(), + installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks(), + installSnapshotState); + installSnapshotState.reset(); } } } + private void sendSnapshotChunk(final ActorSelection followerActor, final FollowerLogInformation followerLogInfo, + final byte[] snapshotChunk, final int chunkIndex, + final Optional serverConfig) { + LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState(); + + installSnapshotState.startChunkTimer(); + followerActor.tell( + new InstallSnapshot(currentTerm(), context.getId(), + snapshotHolder.get().getLastIncludedIndex(), + snapshotHolder.get().getLastIncludedTerm(), + snapshotChunk, + chunkIndex, + installSnapshotState.getTotalChunks(), + OptionalInt.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()); - sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toMillis(), true); + sendAppendEntries(context.getConfigParams().getHeartBeatInterval().toNanos(), true); appendEntriesMessageSlicer.checkExpiredSlicedMessageState(); } @@ -961,7 +1040,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 @@ -1028,7 +1107,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @VisibleForTesting - public FollowerLogInformation getFollower(String followerId) { + public FollowerLogInformation getFollower(final String followerId) { return followerToLog.get(followerId); } @@ -1042,9 +1121,9 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private final long lastIncludedIndex; private final ByteSource snapshotBytes; - SnapshotHolder(Snapshot snapshot, ByteSource snapshotBytes) { - this.lastIncludedTerm = snapshot.getLastAppliedTerm(); - this.lastIncludedIndex = snapshot.getLastAppliedIndex(); + SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) { + lastIncludedTerm = snapshot.getLastAppliedTerm(); + lastIncludedIndex = snapshot.getLastAppliedIndex(); this.snapshotBytes = snapshotBytes; }