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.base.Preconditions;
12 import com.google.common.base.Stopwatch;
13 import java.util.concurrent.TimeUnit;
14 import javax.annotation.Nonnull;
15 import javax.annotation.Nullable;
16 import org.opendaylight.controller.cluster.raft.behaviors.LeaderInstallSnapshotState;
19 * Implementation of the FollowerLogInformation interface.
22 * @author Thomas Pantelis
24 public class FollowerLogInformationImpl implements FollowerLogInformation {
25 private final Stopwatch stopwatch = Stopwatch.createUnstarted();
27 private final RaftActorContext context;
29 private long nextIndex;
31 private long matchIndex;
33 private long lastReplicatedIndex = -1L;
35 private final Stopwatch lastReplicatedStopwatch = Stopwatch.createUnstarted();
37 private short payloadVersion = -1;
39 // Assume the HELIUM_VERSION version initially for backwards compatibility until we obtain the follower's
40 // actual version via AppendEntriesReply. Although we no longer support the Helium version, a pre-Boron
41 // follower will not have the version field in AppendEntriesReply so it will be set to 0 which is
43 private short raftVersion = RaftVersions.HELIUM_VERSION;
45 private final PeerInfo peerInfo;
47 private LeaderInstallSnapshotState installSnapshotState;
49 private long slicedLogEntryIndex = NO_INDEX;
52 * Constructs an instance.
54 * @param peerInfo the associated PeerInfo of the follower.
55 * @param matchIndex the initial match index.
56 * @param context the RaftActorContext.
58 public FollowerLogInformationImpl(final PeerInfo peerInfo, final long matchIndex, final RaftActorContext context) {
59 this.nextIndex = context.getCommitIndex();
60 this.matchIndex = matchIndex;
61 this.context = context;
62 this.peerInfo = Preconditions.checkNotNull(peerInfo);
66 public long incrNextIndex() {
71 public boolean decrNextIndex() {
81 @SuppressWarnings("checkstyle:hiddenField")
82 public boolean setNextIndex(final long nextIndex) {
83 if (this.nextIndex != nextIndex) {
84 this.nextIndex = nextIndex;
92 public long incrMatchIndex() {
97 @SuppressWarnings("checkstyle:hiddenField")
98 public boolean setMatchIndex(final long matchIndex) {
99 // If the new match index is the index of the entry currently being sliced, then we know slicing is complete
100 // and the follower received the entry and responded so clear the slicedLogEntryIndex
101 if (isLogEntrySlicingInProgress() && slicedLogEntryIndex == matchIndex) {
102 slicedLogEntryIndex = NO_INDEX;
105 if (this.matchIndex != matchIndex) {
106 this.matchIndex = matchIndex;
114 public String getId() {
115 return peerInfo.getId();
119 public long getNextIndex() {
124 public long getMatchIndex() {
129 public boolean isFollowerActive() {
130 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
134 long elapsed = stopwatch.elapsed(TimeUnit.MILLISECONDS);
135 return stopwatch.isRunning()
136 && elapsed <= context.getConfigParams().getElectionTimeOutInterval().toMillis();
140 public void markFollowerActive() {
141 if (stopwatch.isRunning()) {
148 public void markFollowerInActive() {
149 if (stopwatch.isRunning()) {
155 public long timeSinceLastActivity() {
156 return stopwatch.elapsed(TimeUnit.MILLISECONDS);
160 public boolean okToReplicate() {
161 if (peerInfo.getVotingState() == VotingState.VOTING_NOT_INITIALIZED) {
165 // Return false if we are trying to send duplicate data before the heartbeat interval
166 if (getNextIndex() == lastReplicatedIndex && lastReplicatedStopwatch.elapsed(TimeUnit.MILLISECONDS)
167 < context.getConfigParams().getHeartBeatInterval().toMillis()) {
171 resetLastReplicated();
175 private void resetLastReplicated() {
176 lastReplicatedIndex = getNextIndex();
177 if (lastReplicatedStopwatch.isRunning()) {
178 lastReplicatedStopwatch.reset();
180 lastReplicatedStopwatch.start();
184 public short getPayloadVersion() {
185 return payloadVersion;
189 public void setPayloadVersion(final short payloadVersion) {
190 this.payloadVersion = payloadVersion;
194 public short getRaftVersion() {
199 public void setRaftVersion(final short raftVersion) {
200 this.raftVersion = raftVersion;
205 public LeaderInstallSnapshotState getInstallSnapshotState() {
206 return installSnapshotState;
210 public void setLeaderInstallSnapshotState(@Nonnull final LeaderInstallSnapshotState state) {
211 if (this.installSnapshotState == null) {
212 this.installSnapshotState = Preconditions.checkNotNull(state);
217 public void clearLeaderInstallSnapshotState() {
218 Preconditions.checkState(installSnapshotState != null);
219 installSnapshotState.close();
220 installSnapshotState = null;
224 public void setSlicedLogEntryIndex(final long index) {
225 slicedLogEntryIndex = index;
229 public boolean isLogEntrySlicingInProgress() {
230 return slicedLogEntryIndex != NO_INDEX;
234 public String toString() {
235 return "FollowerLogInformationImpl [id=" + getId() + ", nextIndex=" + nextIndex + ", matchIndex=" + matchIndex
236 + ", lastReplicatedIndex=" + lastReplicatedIndex + ", votingState=" + peerInfo.getVotingState()
237 + ", stopwatch=" + stopwatch.elapsed(TimeUnit.MILLISECONDS) + ", followerTimeoutMillis="
238 + context.getConfigParams().getElectionTimeOutInterval().toMillis() + "]";