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
9 package org.opendaylight.controller.cluster.raft;
11 import com.google.common.annotations.VisibleForTesting;
12 import com.google.common.base.Preconditions;
13 import com.google.common.base.Stopwatch;
14 import java.util.concurrent.TimeUnit;
15 import javax.annotation.Nonnull;
16 import javax.annotation.Nullable;
17 import org.opendaylight.controller.cluster.raft.behaviors.LeaderInstallSnapshotState;
20 * The state of the followers log as known by the Leader.
23 * @author Thomas Pantelis
25 public final class FollowerLogInformation {
26 public static final long NO_INDEX = -1;
28 private final Stopwatch stopwatch = Stopwatch.createUnstarted();
30 private final RaftActorContext context;
32 private long nextIndex;
34 private long matchIndex;
36 private long lastReplicatedIndex = -1L;
38 private final Stopwatch lastReplicatedStopwatch = Stopwatch.createUnstarted();
40 private short payloadVersion = -1;
42 // Assume the HELIUM_VERSION version initially for backwards compatibility until we obtain the follower's
43 // actual version via AppendEntriesReply. Although we no longer support the Helium version, a pre-Boron
44 // follower will not have the version field in AppendEntriesReply so it will be set to 0 which is
46 private short raftVersion = RaftVersions.HELIUM_VERSION;
48 private final PeerInfo peerInfo;
50 private LeaderInstallSnapshotState installSnapshotState;
52 private long slicedLogEntryIndex = NO_INDEX;
55 * Constructs an instance.
57 * @param peerInfo the associated PeerInfo of the follower.
58 * @param matchIndex the initial match index.
59 * @param context the RaftActorContext.
62 FollowerLogInformation(final PeerInfo peerInfo, final long matchIndex, final RaftActorContext context) {
63 this.nextIndex = context.getCommitIndex();
64 this.matchIndex = matchIndex;
65 this.context = context;
66 this.peerInfo = Preconditions.checkNotNull(peerInfo);
70 * Constructs an instance with no matching index.
72 * @param peerInfo the associated PeerInfo of the follower.
73 * @param context the RaftActorContext.
75 public FollowerLogInformation(final PeerInfo peerInfo, final RaftActorContext context) {
76 this(peerInfo, NO_INDEX, context);
80 * Increments the value of the follower's next index.
82 * @return the new value of nextIndex.
85 long incrNextIndex() {
90 * Decrements the value of the follower's next index.
92 * @return true if the next index was decremented, ie it was previously >= 0, false otherwise.
94 public boolean decrNextIndex() {
104 * Sets the index of the follower's next log entry.
106 * @param nextIndex the new index.
107 * @return true if the new index differed from the current index and the current index was updated, false
110 @SuppressWarnings("checkstyle:hiddenField")
111 public boolean setNextIndex(final long nextIndex) {
112 if (this.nextIndex != nextIndex) {
113 this.nextIndex = nextIndex;
121 * Increments the value of the follower's match index.
123 * @return the new value of matchIndex.
125 public long incrMatchIndex() {
130 * Sets the index of the follower's highest log entry.
132 * @param matchIndex the new index.
133 * @return true if the new index differed from the current index and the current index was updated, false
136 @SuppressWarnings("checkstyle:hiddenField")
137 public boolean setMatchIndex(final long matchIndex) {
138 // If the new match index is the index of the entry currently being sliced, then we know slicing is complete
139 // and the follower received the entry and responded so clear the slicedLogEntryIndex
140 if (isLogEntrySlicingInProgress() && slicedLogEntryIndex == matchIndex) {
141 slicedLogEntryIndex = NO_INDEX;
144 if (this.matchIndex != matchIndex) {
145 this.matchIndex = matchIndex;
153 * Returns the identifier of the follower.
155 * @return the identifier of the follower.
157 public String getId() {
158 return peerInfo.getId();
162 * Returns the index of the next log entry to send to the follower.
164 * @return index of the follower's next log entry.
166 public long getNextIndex() {
171 * Returns the index of highest log entry known to be replicated on the follower.
173 * @return the index of highest log entry.
175 public long getMatchIndex() {
180 * Checks if the follower is active by comparing the time of the last activity with the election time out. The
181 * follower is active if some activity has occurred for the follower within the election time out interval.
183 * @return true if follower is active, false otherwise.
185 public boolean isFollowerActive() {
186 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
190 long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
191 return stopwatch.isRunning()
192 && elapsed <= context.getConfigParams().getElectionTimeOutInterval().toMillis();
196 * Marks the follower as active. This should be called when some activity has occurred for the follower.
198 public void markFollowerActive() {
199 if (stopwatch.isRunning()) {
206 * Marks the follower as inactive. This should only be called from unit tests.
209 public void markFollowerInActive() {
210 if (stopwatch.isRunning()) {
216 * Returns the time since the last activity occurred for the follower.
218 * @return time in nanoseconds since the last activity from the follower.
220 public long nanosSinceLastActivity() {
221 return stopwatch.elapsed(TimeUnit.NANOSECONDS);
225 * This method checks if the next replicate message can be sent to the follower. This is an optimization to avoid
226 * sending duplicate message too frequently if the last replicate message was sent and no reply has been received
227 * yet within the current heart beat interval
229 * @return true if it is OK to replicate, false otherwise
231 public boolean okToReplicate() {
232 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
236 // Return false if we are trying to send duplicate data before the heartbeat interval
237 if (getNextIndex() == lastReplicatedIndex && lastReplicatedStopwatch.elapsed(TimeUnit.MILLISECONDS)
238 < context.getConfigParams().getHeartBeatInterval().toMillis()) {
242 resetLastReplicated();
246 private void resetLastReplicated() {
247 lastReplicatedIndex = getNextIndex();
248 if (lastReplicatedStopwatch.isRunning()) {
249 lastReplicatedStopwatch.reset();
251 lastReplicatedStopwatch.start();
255 * Returns the log entry payload data version of the follower.
257 * @return the payload data version.
259 public short getPayloadVersion() {
260 return payloadVersion;
264 * Sets the payload data version of the follower.
266 * @param payloadVersion the payload data version.
268 public void setPayloadVersion(final short payloadVersion) {
269 this.payloadVersion = payloadVersion;
273 * Returns the the raft version of the follower.
275 * @return the raft version of the follower.
277 public short getRaftVersion() {
282 * Sets the raft version of the follower.
284 * @param raftVersion the raft version.
286 public void setRaftVersion(final short raftVersion) {
287 this.raftVersion = raftVersion;
291 * Returns the LeaderInstallSnapshotState for the in progress install snapshot.
293 * @return the LeaderInstallSnapshotState if a snapshot install is in progress, null otherwise.
296 public LeaderInstallSnapshotState getInstallSnapshotState() {
297 return installSnapshotState;
301 * Sets the LeaderInstallSnapshotState when an install snapshot is initiated.
303 * @param state the LeaderInstallSnapshotState
305 public void setLeaderInstallSnapshotState(@Nonnull final LeaderInstallSnapshotState state) {
306 if (this.installSnapshotState == null) {
307 this.installSnapshotState = Preconditions.checkNotNull(state);
312 * Clears the LeaderInstallSnapshotState when an install snapshot is complete.
314 public void clearLeaderInstallSnapshotState() {
315 Preconditions.checkState(installSnapshotState != null);
316 installSnapshotState.close();
317 installSnapshotState = null;
321 * Sets the index of the log entry whose payload size exceeds the maximum size for a single message and thus
322 * needs to be sliced into smaller chunks.
324 * @param index the log entry index or NO_INDEX to clear it
326 public void setSlicedLogEntryIndex(final long index) {
327 slicedLogEntryIndex = index;
331 * Return whether or not log entry slicing is currently in progress.
333 * @return true if slicing is currently in progress, false otherwise
335 public boolean isLogEntrySlicingInProgress() {
336 return slicedLogEntryIndex != NO_INDEX;
340 public String toString() {
341 return "FollowerLogInformation [id=" + getId() + ", nextIndex=" + nextIndex + ", matchIndex=" + matchIndex
342 + ", lastReplicatedIndex=" + lastReplicatedIndex + ", votingState=" + peerInfo.getVotingState()
343 + ", stopwatch=" + stopwatch.elapsed(TimeUnit.MILLISECONDS) + ", followerTimeoutMillis="
344 + context.getConfigParams().getElectionTimeOutInterval().toMillis() + "]";