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=fef6cc8b7c57a142d683d1e0f27214ad751fd9d3;hb=86e8e4a06b682aa772c834a2cef56d0596540e1b;hpb=b4bf55727093657662d8c16a50fa85f87978a586 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 fef6cc8b7c..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 @@ -13,7 +13,6 @@ 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.io.ByteSource; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; @@ -26,6 +25,8 @@ 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 org.eclipse.jdt.annotation.Nullable; @@ -40,14 +41,17 @@ 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; @@ -98,7 +102,7 @@ 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(final RaftActorContext context, final RaftState state, @@ -162,7 +166,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerToLog.remove(followerId); } - public void updateMinReplicaCount() { + public final void updateMinReplicaCount() { int numVoting = 0; for (PeerInfo peer: context.getPeers()) { if (peer.isVoting()) { @@ -186,7 +190,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { @VisibleForTesting void setSnapshotHolder(final @Nullable SnapshotHolder snapshotHolder) { - this.snapshotHolder = Optional.fromNullable(snapshotHolder); + this.snapshotHolder = Optional.ofNullable(snapshotHolder); } @VisibleForTesting @@ -432,8 +436,12 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { super.performSnapshotWithoutCapture(minReplicatedToAllIndex); } - @Override - protected ClientRequestTracker removeClientRequestTracker(final 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(); @@ -447,12 +455,35 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @Override - protected RaftActorBehavior handleRequestVoteReply(final ActorRef sender, - final 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(final ActorRef sender, final Object message) { @@ -462,12 +493,11 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { 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()); @@ -481,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); @@ -494,8 +524,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { beforeSendHeartbeat(); sendHeartBeat(); scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval()); - } else if (message instanceof SendInstallSnapshot) { - SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message; + } else if (message instanceof SendInstallSnapshot sendInstallSnapshot) { setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes())); sendInstallSnapshot(); @@ -533,6 +562,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return; } + installSnapshotState.resetChunkTimer(); followerLogInformation.markFollowerActive(); if (installSnapshotState.getChunkIndex() == reply.getChunkIndex()) { @@ -664,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; } @@ -688,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; } @@ -716,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; @@ -742,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; } @@ -827,6 +866,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { appendEntries); } + followerLogInformation.setSentCommitIndex(leaderCommitIndex); followerActor.tell(appendEntries, actor()); } @@ -862,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())); @@ -926,31 +966,63 @@ 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(), + sendSnapshotChunk(followerActor, followerLogInfo, nextSnapshotChunk, nextChunkIndex, serverConfig); + + log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(), + installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks()); + + } catch (IOException 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(), - nextSnapshotChunk, - nextChunkIndex, + snapshotChunk, + chunkIndex, installSnapshotState.getTotalChunks(), - Optional.of(installSnapshotState.getLastChunkHashCode()), + OptionalInt.of(installSnapshotState.getLastChunkHashCode()), serverConfig - ).toSerializable(followerLogInfo.getRaftVersion()), - actor() - ); - - } catch (IOException e) { - throw new RuntimeException(e); - } + ).toSerializable(followerLogInfo.getRaftVersion()), + actor() + ); + } - log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(), - installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks()); + 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() { @@ -1050,8 +1122,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private final ByteSource snapshotBytes; SnapshotHolder(final Snapshot snapshot, final ByteSource snapshotBytes) { - this.lastIncludedTerm = snapshot.getLastAppliedTerm(); - this.lastIncludedIndex = snapshot.getLastAppliedIndex(); + lastIncludedTerm = snapshot.getLastAppliedTerm(); + lastIncludedIndex = snapshot.getLastAppliedIndex(); this.snapshotBytes = snapshotBytes; }