X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2Fbehaviors%2FAbstractLeader.java;h=fef6cc8b7c57a142d683d1e0f27214ad751fd9d3;hb=refs%2Fchanges%2F11%2F80211%2F6;hp=9fc7b3393cbe1f86404e2eb9a08a9ca6f6e21bc1;hpb=8d90cf04be86f872f7eeb892d37517d5ee087157;p=controller.git 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..fef6cc8b7c 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,17 @@ * 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; @@ -28,14 +28,13 @@ import java.util.Map; import java.util.Map.Entry; 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; @@ -58,7 +57,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: @@ -102,8 +101,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { private Optional snapshotHolder = Optional.absent(); 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 +116,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 +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); } @@ -149,9 +148,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,7 +158,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } } - public void removeFollower(String followerId) { + public void removeFollower(final String followerId) { followerToLog.remove(followerId); } @@ -186,7 +185,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @VisibleForTesting - void setSnapshot(@Nullable SnapshotHolder snapshotHolder) { + void setSnapshotHolder(final @Nullable SnapshotHolder snapshotHolder) { this.snapshotHolder = Optional.fromNullable(snapshotHolder); } @@ -196,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); @@ -205,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 @@ -217,20 +217,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 +247,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 +259,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 +281,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 +295,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 +405,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; @@ -431,7 +433,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @Override - protected ClientRequestTracker removeClientRequestTracker(long logIndex) { + protected ClientRequestTracker removeClientRequestTracker(final long logIndex) { final Iterator it = trackers.iterator(); while (it.hasNext()) { final ClientRequestTracker t = it.next(); @@ -445,16 +447,16 @@ 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) { - 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; @@ -494,7 +496,8 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { scheduleHeartBeat(context.getConfigParams().getHeartBeatInterval()); } else if (message instanceof SendInstallSnapshot) { SendInstallSnapshot sendInstallSnapshot = (SendInstallSnapshot) message; - setSnapshot(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), sendInstallSnapshot.getSnapshotBytes())); + setSnapshotHolder(new SnapshotHolder(sendInstallSnapshot.getSnapshot(), + sendInstallSnapshot.getSnapshotBytes())); sendInstallSnapshot(); } else if (message instanceof Replicate) { replicate((Replicate) message); @@ -509,7 +512,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(); @@ -548,7 +553,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 +608,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 +635,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 +652,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) { @@ -725,8 +730,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(); @@ -795,8 +800,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,7 +819,8 @@ 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(), @@ -843,7 +849,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 @@ -865,7 +871,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 +903,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) { @@ -938,18 +944,19 @@ 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()); } } 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 +968,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 +1035,7 @@ public abstract class AbstractLeader extends AbstractRaftActorBehavior { } @VisibleForTesting - public FollowerLogInformation getFollower(String followerId) { + public FollowerLogInformation getFollower(final String followerId) { return followerToLog.get(followerId); } @@ -1042,7 +1049,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;