2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
4 * This program and the accompanying materials are made available under the
5 * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6 * and is available at http://www.eclipse.org/legal/epl-v10.html
8 package org.opendaylight.controller.cluster.raft;
10 import static com.google.common.base.Preconditions.checkState;
11 import static java.util.Objects.requireNonNull;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.common.base.Stopwatch;
15 import java.util.concurrent.TimeUnit;
16 import org.eclipse.jdt.annotation.NonNull;
17 import org.eclipse.jdt.annotation.Nullable;
18 import org.opendaylight.controller.cluster.raft.behaviors.LeaderInstallSnapshotState;
21 * The state of the followers log as known by the Leader.
24 * @author Thomas Pantelis
26 public final class FollowerLogInformation {
27 public static final long NO_INDEX = -1;
29 private final Stopwatch stopwatch = Stopwatch.createUnstarted();
31 private final RaftActorContext context;
33 private long nextIndex;
35 private long matchIndex;
37 private long lastReplicatedIndex = -1L;
39 private long sentCommitIndex = -1L;
41 private final Stopwatch lastReplicatedStopwatch = Stopwatch.createUnstarted();
43 private short payloadVersion = -1;
45 // Assume the HELIUM_VERSION version initially for backwards compatibility until we obtain the follower's
46 // actual version via AppendEntriesReply. Although we no longer support the Helium version, a pre-Boron
47 // follower will not have the version field in AppendEntriesReply so it will be set to 0 which is
49 private short raftVersion = RaftVersions.HELIUM_VERSION;
51 private final PeerInfo peerInfo;
53 private LeaderInstallSnapshotState installSnapshotState;
55 private long slicedLogEntryIndex = NO_INDEX;
57 private boolean needsLeaderAddress;
60 * Constructs an instance.
62 * @param peerInfo the associated PeerInfo of the follower.
63 * @param matchIndex the initial match index.
64 * @param context the RaftActorContext.
67 FollowerLogInformation(final PeerInfo peerInfo, final long matchIndex, final RaftActorContext context) {
68 this.nextIndex = context.getCommitIndex();
69 this.matchIndex = matchIndex;
70 this.context = context;
71 this.peerInfo = requireNonNull(peerInfo);
75 * Constructs an instance with no matching index.
77 * @param peerInfo the associated PeerInfo of the follower.
78 * @param context the RaftActorContext.
80 public FollowerLogInformation(final PeerInfo peerInfo, final RaftActorContext context) {
81 this(peerInfo, NO_INDEX, context);
85 * Increments the value of the follower's next index.
87 * @return the new value of nextIndex.
90 long incrNextIndex() {
95 * Decrements the value of the follower's next index, taking into account its reported last log index.
97 * @param followerLastIndex follower's last reported index.
98 * @return true if the next index was decremented, i.e. it was previously >= 0, false otherwise.
100 public boolean decrNextIndex(final long followerLastIndex) {
105 if (followerLastIndex >= 0 && nextIndex > followerLastIndex) {
106 // If the follower's last log index is lower than nextIndex, jump directly to it, so we converge
107 // on a common index more quickly.
108 nextIndex = followerLastIndex;
116 * Sets the index of the follower's next log entry.
118 * @param nextIndex the new index.
119 * @return true if the new index differed from the current index and the current index was updated, false
122 @SuppressWarnings("checkstyle:hiddenField")
123 public boolean setNextIndex(final long nextIndex) {
124 if (this.nextIndex != nextIndex) {
125 this.nextIndex = nextIndex;
133 * Increments the value of the follower's match index.
135 * @return the new value of matchIndex.
137 public long incrMatchIndex() {
142 * Sets the index of the follower's highest log entry.
144 * @param matchIndex the new index.
145 * @return true if the new index differed from the current index and the current index was updated, false
148 @SuppressWarnings("checkstyle:hiddenField")
149 public boolean setMatchIndex(final long matchIndex) {
150 // If the new match index is the index of the entry currently being sliced, then we know slicing is complete
151 // and the follower received the entry and responded so clear the slicedLogEntryIndex
152 if (isLogEntrySlicingInProgress() && slicedLogEntryIndex == matchIndex) {
153 slicedLogEntryIndex = NO_INDEX;
156 if (this.matchIndex != matchIndex) {
157 this.matchIndex = matchIndex;
165 * Returns the identifier of the follower.
167 * @return the identifier of the follower.
169 public String getId() {
170 return peerInfo.getId();
174 * Returns the index of the next log entry to send to the follower.
176 * @return index of the follower's next log entry.
178 public long getNextIndex() {
183 * Returns the index of highest log entry known to be replicated on the follower.
185 * @return the index of highest log entry.
187 public long getMatchIndex() {
192 * Checks if the follower is active by comparing the time of the last activity with the election time out. The
193 * follower is active if some activity has occurred for the follower within the election time out interval.
195 * @return true if follower is active, false otherwise.
197 public boolean isFollowerActive() {
198 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
202 long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
203 return stopwatch.isRunning()
204 && elapsed <= context.getConfigParams().getElectionTimeOutInterval().toMillis();
208 * Marks the follower as active. This should be called when some activity has occurred for the follower.
210 public void markFollowerActive() {
211 if (stopwatch.isRunning()) {
218 * Marks the follower as inactive. This should only be called from unit tests.
221 public void markFollowerInActive() {
222 if (stopwatch.isRunning()) {
228 * Returns the time since the last activity occurred for the follower.
230 * @return time in nanoseconds since the last activity from the follower.
232 public long nanosSinceLastActivity() {
233 return stopwatch.elapsed(TimeUnit.NANOSECONDS);
237 * This method checks if the next replicate message can be sent to the follower. This is an optimization to avoid
238 * sending duplicate message too frequently if the last replicate message was sent and no reply has been received
239 * yet within the current heart beat interval
241 * @param commitIndex current commitIndex
242 * @return true if it is OK to replicate, false otherwise
244 public boolean okToReplicate(final long commitIndex) {
245 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
249 // Return false if we are trying to send duplicate data before the heartbeat interval. This check includes
250 // also our commitIndex, as followers need to be told of new commitIndex as soon as possible.
251 if (getNextIndex() == lastReplicatedIndex && !hasStaleCommitIndex(commitIndex)
252 && lastReplicatedStopwatch.elapsed(TimeUnit.MILLISECONDS)
253 < context.getConfigParams().getHeartBeatInterval().toMillis()) {
257 resetLastReplicated();
261 private void resetLastReplicated() {
262 lastReplicatedIndex = getNextIndex();
263 if (lastReplicatedStopwatch.isRunning()) {
264 lastReplicatedStopwatch.reset();
266 lastReplicatedStopwatch.start();
270 * Returns the log entry payload data version of the follower.
272 * @return the payload data version.
274 public short getPayloadVersion() {
275 return payloadVersion;
279 * Sets the payload data version of the follower.
281 * @param payloadVersion the payload data version.
283 public void setPayloadVersion(final short payloadVersion) {
284 this.payloadVersion = payloadVersion;
288 * Returns the the raft version of the follower.
290 * @return the raft version of the follower.
292 public short getRaftVersion() {
297 * Sets the raft version of the follower.
299 * @param raftVersion the raft version.
301 public void setRaftVersion(final short raftVersion) {
302 this.raftVersion = raftVersion;
306 * Returns the LeaderInstallSnapshotState for the in progress install snapshot.
308 * @return the LeaderInstallSnapshotState if a snapshot install is in progress, null otherwise.
310 public @Nullable LeaderInstallSnapshotState getInstallSnapshotState() {
311 return installSnapshotState;
315 * Sets the LeaderInstallSnapshotState when an install snapshot is initiated.
317 * @param state the LeaderInstallSnapshotState
319 public void setLeaderInstallSnapshotState(final @NonNull LeaderInstallSnapshotState state) {
320 if (this.installSnapshotState == null) {
321 this.installSnapshotState = requireNonNull(state);
326 * Clears the LeaderInstallSnapshotState when an install snapshot is complete.
328 public void clearLeaderInstallSnapshotState() {
329 checkState(installSnapshotState != null);
330 installSnapshotState.close();
331 installSnapshotState = null;
335 * Sets the index of the log entry whose payload size exceeds the maximum size for a single message and thus
336 * needs to be sliced into smaller chunks.
338 * @param index the log entry index or NO_INDEX to clear it
340 public void setSlicedLogEntryIndex(final long index) {
341 slicedLogEntryIndex = index;
345 * Return whether or not log entry slicing is currently in progress.
347 * @return true if slicing is currently in progress, false otherwise
349 public boolean isLogEntrySlicingInProgress() {
350 return slicedLogEntryIndex != NO_INDEX;
353 public void setNeedsLeaderAddress(final boolean value) {
354 needsLeaderAddress = value;
357 public @Nullable String needsLeaderAddress(final String leaderId) {
358 return needsLeaderAddress ? context.getPeerAddress(leaderId) : null;
361 public boolean hasStaleCommitIndex(final long commitIndex) {
362 return sentCommitIndex != commitIndex;
365 public void setSentCommitIndex(final long commitIndex) {
366 sentCommitIndex = commitIndex;
370 public String toString() {
371 return "FollowerLogInformation [id=" + getId() + ", nextIndex=" + nextIndex + ", matchIndex=" + matchIndex
372 + ", lastReplicatedIndex=" + lastReplicatedIndex + ", commitIndex=" + sentCommitIndex
373 + ", votingState=" + peerInfo.getVotingState()
374 + ", stopwatch=" + stopwatch.elapsed(TimeUnit.MILLISECONDS)
375 + ", followerTimeoutMillis=" + context.getConfigParams().getElectionTimeOutInterval().toMillis() + "]";