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=6560ad76c3937285300173f16df53f4c60d9ae1d;hp=ec465935039eb99a0c34341e05aa93bd2340e34d;hb=HEAD;hpb=f33beecf2a10955a9219757529ba3017079816cc 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 ec46593503..7514dccff4 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; @@ -21,11 +20,12 @@ import java.io.ObjectOutputStream; import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.Iterator; 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; @@ -33,19 +33,21 @@ 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.PeerInfo; import org.opendaylight.controller.cluster.raft.RaftActorContext; import org.opendaylight.controller.cluster.raft.RaftState; +import org.opendaylight.controller.cluster.raft.RaftVersions; 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.RaftRPC; @@ -98,7 +100,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, @@ -106,7 +108,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { super(context, state); appendEntriesMessageSlicer = MessageSlicer.builder().logContext(logName()) - .messageSliceSize(context.getConfigParams().getSnapshotChunkSize()) + .messageSliceSize(context.getConfigParams().getMaximumMessageSliceSize()) .expireStateAfterInactivity(context.getConfigParams().getElectionTimeOutInterval().toMillis() * 3, TimeUnit.MILLISECONDS).build(); @@ -162,7 +164,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 +188,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 @@ -217,6 +219,13 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } + final var followerRaftVersion = appendEntriesReply.getRaftVersion(); + if (followerRaftVersion < RaftVersions.FLUORINE_VERSION) { + log.warn("{}: handleAppendEntriesReply - ignoring reply from follower {} raft version {}", logName(), + followerId, followerRaftVersion); + return this; + } + final long lastActivityNanos = followerLogInformation.nanosSinceLastActivity(); if (lastActivityNanos > context.getConfigParams().getElectionTimeOutInterval().toNanos()) { log.warn("{} : handleAppendEntriesReply delayed beyond election timeout, " @@ -227,7 +236,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { followerLogInformation.markFollowerActive(); followerLogInformation.setPayloadVersion(appendEntriesReply.getPayloadVersion()); - followerLogInformation.setRaftVersion(appendEntriesReply.getRaftVersion()); + followerLogInformation.setRaftVersion(followerRaftVersion); followerLogInformation.setNeedsLeaderAddress(appendEntriesReply.isNeedsLeaderAddress()); long followerLastLogIndex = appendEntriesReply.getLogLastIndex(); @@ -432,27 +441,52 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { super.performSnapshotWithoutCapture(minReplicatedToAllIndex); } - @Override - protected ClientRequestTracker removeClientRequestTracker(final long logIndex) { - final Iterator it = trackers.iterator(); + /** + * 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 var it = trackers.iterator(); while (it.hasNext()) { - final ClientRequestTracker t = it.next(); - if (t.getIndex() == logIndex) { + final var tracker = it.next(); + if (tracker.logIndex() == logIndex) { it.remove(); - return t; + return tracker; } } - return null; } @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 var tracker = removeClientRequestTracker(entry.getIndex()); + if (tracker != null) { + return new ApplyState(tracker.clientActor(), tracker.identifier(), 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. + if (entry.getData() instanceof IdentifiablePayload identifiable) { + return new ApplyState(null, identifiable.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,47 +496,45 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { return this; } - if (message instanceof RaftRPC) { - RaftRPC rpc = (RaftRPC) message; - // 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()) { - log.info("{}: Term {} in \"{}\" message is greater than leader's term {} - switching to Follower", - logName(), rpc.getTerm(), rpc, context.getTermInformation().getCurrentTerm()); - - context.getTermInformation().updateAndPersist(rpc.getTerm(), null); - - // This is a special case. Normally when stepping down as leader we don't process and reply to the - // RaftRPC as per raft. But if we're in the process of transferring leadership and we get a - // RequestVote, process the RequestVote before switching to Follower. This enables the requesting - // candidate node to be elected the leader faster and avoids us possibly timing out in the Follower - // state and starting a new election and grabbing leadership back before the other candidate node can - // 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) { - log.debug("{}: Leadership transfer in progress - processing RequestVote", logName()); - super.handleMessage(sender, message); - } - - return internalSwitchBehavior(RaftState.Follower); + // 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 (message instanceof RaftRPC rpc && 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()); + + context.getTermInformation().updateAndPersist(rpc.getTerm(), null); + + // This is a special case. Normally when stepping down as leader we don't process and reply to the + // RaftRPC as per raft. But if we're in the process of transferring leadership and we get a + // RequestVote, process the RequestVote before switching to Follower. This enables the requesting + // candidate node to be elected the leader faster and avoids us possibly timing out in the Follower + // state and starting a new election and grabbing leadership back before the other candidate node can + // 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 (rpc instanceof RequestVote requestVote && context.getRaftActorLeadershipTransferCohort() != null) { + log.debug("{}: Leadership transfer in progress - processing RequestVote", logName()); + requestVote(sender, requestVote); } + + return internalSwitchBehavior(RaftState.Follower); } if (message instanceof SendHeartBeat) { 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(); - } else if (message instanceof Replicate) { - replicate((Replicate) message); - } else if (message instanceof InstallSnapshotReply) { - handleInstallSnapshotReply((InstallSnapshotReply) message); + } else if (message instanceof Replicate replicate) { + replicate(replicate); + } else if (message instanceof InstallSnapshotReply installSnapshotReply) { + handleInstallSnapshotReply(installSnapshotReply); } else if (message instanceof CheckConsensusReached) { possiblyUpdateCommitIndex(); } else { @@ -542,7 +574,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { if (installSnapshotState.isLastChunk(reply.getChunkIndex())) { //this was the last chunk reply - long followerMatchIndex = snapshotHolder.get().getLastIncludedIndex(); + long followerMatchIndex = snapshotHolder.orElseThrow().getLastIncludedIndex(); followerLogInformation.setMatchIndex(followerMatchIndex); followerLogInformation.setNextIndex(followerMatchIndex + 1); followerLogInformation.clearLeaderInstallSnapshotState(); @@ -610,17 +642,16 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } private void replicate(final Replicate replicate) { - long logIndex = replicate.getReplicatedLogEntry().getIndex(); + final long logIndex = replicate.logIndex(); - log.debug("{}: Replicate message: identifier: {}, logIndex: {}, payload: {}, isSendImmediate: {}", logName(), - replicate.getIdentifier(), logIndex, replicate.getReplicatedLogEntry().getData().getClass(), - replicate.isSendImmediate()); + log.debug("{}: Replicate message: identifier: {}, logIndex: {}, isSendImmediate: {}", logName(), + replicate.identifier(), logIndex, replicate.sendImmediate()); // Create a tracker entry we will use this later to notify the // client actor - if (replicate.getClientActor() != null) { - trackers.add(new ClientRequestTrackerImpl(replicate.getClientActor(), replicate.getIdentifier(), - logIndex)); + final var clientActor = replicate.clientActor(); + if (clientActor != null) { + trackers.add(new ClientRequestTracker(logIndex, clientActor, replicate.identifier())); } boolean applyModificationToState = !context.anyVotingPeers() @@ -631,7 +662,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { applyLogToStateMachine(logIndex); } - if (replicate.isSendImmediate() && !followerToLog.isEmpty()) { + if (replicate.sendImmediate() && !followerToLog.isEmpty()) { sendAppendEntries(0, false); } } @@ -677,7 +708,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } else if (installSnapshotState.canSendNextChunk()) { sendSnapshotChunk(followerActor, followerLogInformation); } - } else if (sendHeartbeat) { + } 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; } @@ -698,7 +729,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; } @@ -726,7 +757,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; @@ -745,14 +776,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { // 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 int maxDataSize = context.getConfigParams().getMaximumMessageSliceSize(); final long followerNextIndex = followerLogInfo.getNextIndex(); List 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)) { + if (entries.size() != 1 || entries.get(0).getData().serializedSize() <= maxDataSize) { // Don't need to slice. return entries; } @@ -837,6 +868,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { appendEntries); } + followerLogInformation.setSentCommitIndex(leaderCommitIndex); followerActor.tell(appendEntries, actor()); } @@ -872,10 +904,10 @@ 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())); + context.getConfigParams().getMaximumMessageSliceSize(), logName())); } return captureInitiated; @@ -917,14 +949,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { if (snapshotHolder.isPresent()) { LeaderInstallSnapshotState installSnapshotState = followerLogInfo.getInstallSnapshotState(); if (installSnapshotState == null) { - installSnapshotState = new LeaderInstallSnapshotState(context.getConfigParams().getSnapshotChunkSize(), - logName()); + installSnapshotState = new LeaderInstallSnapshotState( + context.getConfigParams().getMaximumMessageSliceSize(), logName()); followerLogInfo.setLeaderInstallSnapshotState(installSnapshotState); } try { // Ensure the snapshot bytes are set - this is a no-op. - installSnapshotState.setSnapshotBytes(snapshotHolder.get().getSnapshotBytes()); + installSnapshotState.setSnapshotBytes(snapshotHolder.orElseThrow().getSnapshotBytes()); if (!installSnapshotState.canSendNextChunk()) { return; @@ -936,19 +968,22 @@ 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)); } sendSnapshotChunk(followerActor, followerLogInfo, nextSnapshotChunk, nextChunkIndex, serverConfig); + log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(), + installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks()); + } catch (IOException e) { - throw new RuntimeException(e); + log.warn("{}: Unable to send chunk: {}/{}. Reseting snapshot progress. Snapshot state: {}", logName(), + installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks(), + installSnapshotState, e); + installSnapshotState.reset(); } - - log.debug("{}: InstallSnapshot sent to follower {}, Chunk: {}/{}", logName(), followerActor.path(), - installSnapshotState.getChunkIndex(), installSnapshotState.getTotalChunks()); } } @@ -960,14 +995,14 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { installSnapshotState.startChunkTimer(); followerActor.tell( new InstallSnapshot(currentTerm(), context.getId(), - snapshotHolder.get().getLastIncludedIndex(), - snapshotHolder.get().getLastIncludedTerm(), + snapshotHolder.orElseThrow().getLastIncludedIndex(), + snapshotHolder.orElseThrow().getLastIncludedTerm(), snapshotChunk, chunkIndex, installSnapshotState.getTotalChunks(), - Optional.of(installSnapshotState.getLastChunkHashCode()), - serverConfig - ).toSerializable(followerLogInfo.getRaftVersion()), + OptionalInt.of(installSnapshotState.getLastChunkHashCode()), + serverConfig, + followerLogInfo.getRaftVersion()), actor() ); } @@ -1089,8 +1124,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; }