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.behaviors;
11 import akka.actor.ActorRef;
12 import akka.actor.Cancellable;
13 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
14 import org.opendaylight.controller.cluster.raft.RaftActorContext;
15 import org.opendaylight.controller.cluster.raft.RaftState;
16 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
17 import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
18 import org.opendaylight.controller.cluster.raft.internal.messages.ElectionTimeout;
19 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
20 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
21 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
22 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
23 import scala.concurrent.duration.FiniteDuration;
25 import java.util.Random;
26 import java.util.concurrent.TimeUnit;
29 * Abstract class that represents the behavior of a RaftActor
33 * <li> If commitIndex > lastApplied: increment lastApplied, apply
34 * log[lastApplied] to state machine (§5.3)
35 * <li> If RPC request or response contains term T > currentTerm:
36 * set currentTerm = T, convert to follower (§5.1)
38 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
41 * Information about the RaftActor whose behavior this class represents
43 protected final RaftActorContext context;
46 * The maximum election time variance
48 private static final int ELECTION_TIME_MAX_VARIANCE = 100;
51 * The interval at which a heart beat message will be sent to the remote
54 * Since this is set to 100 milliseconds the Election timeout should be
55 * at least 200 milliseconds
57 protected static final FiniteDuration HEART_BEAT_INTERVAL =
58 new FiniteDuration(100, TimeUnit.MILLISECONDS);
61 * The interval in which a new election would get triggered if no leader is found
63 private static final long ELECTION_TIME_INTERVAL =
64 HEART_BEAT_INTERVAL.toMillis() * 2;
69 private Cancellable electionCancel = null;
74 protected String leaderId = null;
77 protected AbstractRaftActorBehavior(RaftActorContext context) {
78 this.context = context;
82 * Derived classes should not directly handle AppendEntries messages it
83 * should let the base class handle it first. Once the base class handles
84 * the AppendEntries message and does the common actions that are applicable
85 * in all RaftState's it will delegate the handling of the AppendEntries
86 * message to the derived class to do more state specific handling by calling
89 * @param sender The actor that sent this message
90 * @param appendEntries The AppendEntries message
93 protected abstract RaftState handleAppendEntries(ActorRef sender,
94 AppendEntries appendEntries);
98 * appendEntries first processes the AppendEntries message and then
99 * delegates handling to a specific behavior
102 * @param appendEntries
105 protected RaftState appendEntries(ActorRef sender,
106 AppendEntries appendEntries) {
108 // 1. Reply false if term < currentTerm (§5.1)
109 if (appendEntries.getTerm() < currentTerm()) {
110 context.getLogger().debug(
111 "Cannot append entries because sender term " + appendEntries
112 .getTerm() + " is less than " + currentTerm());
114 new AppendEntriesReply(context.getId(), currentTerm(), false,
115 lastIndex(), lastTerm()), actor()
121 return handleAppendEntries(sender, appendEntries);
125 * Derived classes should not directly handle AppendEntriesReply messages it
126 * should let the base class handle it first. Once the base class handles
127 * the AppendEntriesReply message and does the common actions that are
128 * applicable in all RaftState's it will delegate the handling of the
129 * AppendEntriesReply message to the derived class to do more state specific
130 * handling by calling this method
132 * @param sender The actor that sent this message
133 * @param appendEntriesReply The AppendEntriesReply message
136 protected abstract RaftState handleAppendEntriesReply(ActorRef sender,
137 AppendEntriesReply appendEntriesReply);
140 * requestVote handles the RequestVote message. This logic is common
147 protected RaftState requestVote(ActorRef sender,
148 RequestVote requestVote) {
150 boolean grantVote = false;
152 // Reply false if term < currentTerm (§5.1)
153 if (requestVote.getTerm() < currentTerm()) {
156 // If votedFor is null or candidateId, and candidate’s log is at
157 // least as up-to-date as receiver’s log, grant vote (§5.2, §5.4)
158 } else if (votedFor() == null || votedFor()
159 .equals(requestVote.getCandidateId())) {
161 boolean candidateLatest = false;
164 // Raft determines which of two logs is more up-to-date
165 // by comparing the index and term of the last entries in the
166 // logs. If the logs have last entries with different terms, then
167 // the log with the later term is more up-to-date. If the logs
168 // end with the same term, then whichever log is longer is
170 if (requestVote.getLastLogTerm() > lastTerm()) {
171 candidateLatest = true;
172 } else if ((requestVote.getLastLogTerm() == lastTerm())
173 && requestVote.getLastLogIndex() >= lastIndex()) {
174 candidateLatest = true;
177 if (candidateLatest) {
179 context.getTermInformation().updateAndPersist(requestVote.getTerm(),
180 requestVote.getCandidateId());
184 sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor());
190 * Derived classes should not directly handle RequestVoteReply messages it
191 * should let the base class handle it first. Once the base class handles
192 * the RequestVoteReply message and does the common actions that are
193 * applicable in all RaftState's it will delegate the handling of the
194 * RequestVoteReply message to the derived class to do more state specific
195 * handling by calling this method
197 * @param sender The actor that sent this message
198 * @param requestVoteReply The RequestVoteReply message
201 protected abstract RaftState handleRequestVoteReply(ActorRef sender,
202 RequestVoteReply requestVoteReply);
205 * Creates a random election duration
209 protected FiniteDuration electionDuration() {
210 long variance = new Random().nextInt(ELECTION_TIME_MAX_VARIANCE);
211 return new FiniteDuration(ELECTION_TIME_INTERVAL + variance,
212 TimeUnit.MILLISECONDS);
216 * stop the scheduled election
218 protected void stopElection() {
219 if (electionCancel != null && !electionCancel.isCancelled()) {
220 electionCancel.cancel();
225 * schedule a new election
229 protected void scheduleElection(FiniteDuration interval) {
232 // Schedule an election. When the scheduler triggers an ElectionTimeout
233 // message is sent to itself
235 context.getActorSystem().scheduler().scheduleOnce(interval,
236 context.getActor(), new ElectionTimeout(),
237 context.getActorSystem().dispatcher(), context.getActor());
241 * Get the current term
244 protected long currentTerm() {
245 return context.getTermInformation().getCurrentTerm();
249 * Get the candidate for whom we voted in the current term
252 protected String votedFor() {
253 return context.getTermInformation().getVotedFor();
257 * Get the actor associated with this behavior
260 protected ActorRef actor() {
261 return context.getActor();
265 * Get the term from the last entry in the log
269 protected long lastTerm() {
270 return context.getReplicatedLog().lastTerm();
274 * Get the index from the last entry in the log
278 protected long lastIndex() {
279 return context.getReplicatedLog().lastIndex();
283 * Find the client request tracker for a specific logIndex
288 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
293 * Find the log index from the previous to last entry in the log
297 protected long prevLogIndex(long index){
298 ReplicatedLogEntry prevEntry =
299 context.getReplicatedLog().get(index - 1);
300 if (prevEntry != null) {
301 return prevEntry.getIndex();
307 * Find the log term from the previous to last entry in the log
310 protected long prevLogTerm(long index){
311 ReplicatedLogEntry prevEntry =
312 context.getReplicatedLog().get(index - 1);
313 if (prevEntry != null) {
314 return prevEntry.getTerm();
320 * Apply the provided index to the state machine
322 * @param index a log index that is known to be committed
324 protected void applyLogToStateMachine(long index) {
325 // Now maybe we apply to the state machine
326 for (long i = context.getLastApplied() + 1;
327 i < index + 1; i++) {
328 ActorRef clientActor = null;
329 String identifier = null;
330 ClientRequestTracker tracker = findClientRequestTracker(i);
332 if (tracker != null) {
333 clientActor = tracker.getClientActor();
334 identifier = tracker.getIdentifier();
336 ReplicatedLogEntry replicatedLogEntry =
337 context.getReplicatedLog().get(i);
339 if (replicatedLogEntry != null) {
340 actor().tell(new ApplyState(clientActor, identifier,
341 replicatedLogEntry), actor());
343 context.getLogger().error(
344 "Missing index " + i + " from log. Cannot apply state.");
347 // Send a local message to the local RaftActor (it's derived class to be
348 // specific to apply the log to it's index)
349 context.setLastApplied(index);
353 public RaftState handleMessage(ActorRef sender, Object message) {
354 if (message instanceof AppendEntries) {
355 return appendEntries(sender, (AppendEntries) message);
356 } else if (message instanceof AppendEntriesReply) {
357 return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
358 } else if (message instanceof RequestVote) {
359 return requestVote(sender, (RequestVote) message);
360 } else if (message instanceof RequestVoteReply) {
361 return handleRequestVoteReply(sender, (RequestVoteReply) message);
366 @Override public String getLeaderId() {