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 java.util.HashMap;
15 import java.util.Random;
16 import java.util.concurrent.TimeUnit;
17 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
18 import org.opendaylight.controller.cluster.raft.RaftActorContext;
19 import org.opendaylight.controller.cluster.raft.RaftState;
20 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
21 import org.opendaylight.controller.cluster.raft.SerializationUtils;
22 import org.opendaylight.controller.cluster.raft.ServerConfigurationPayload;
23 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
24 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
25 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
26 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
27 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
28 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
29 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
30 import org.slf4j.Logger;
31 import scala.concurrent.duration.FiniteDuration;
34 * Abstract class that represents the behavior of a RaftActor
38 * <li> If commitIndex > lastApplied: increment lastApplied, apply
39 * log[lastApplied] to state machine (§5.3)
40 * <li> If RPC request or response contains term T > currentTerm:
41 * set currentTerm = T, convert to follower (§5.1)
43 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
45 protected static final ElectionTimeout ELECTION_TIMEOUT = new ElectionTimeout();
48 * Information about the RaftActor whose behavior this class represents
50 protected final RaftActorContext context;
55 protected final Logger LOG;
60 private Cancellable electionCancel = null;
65 protected String leaderId = null;
67 private short leaderPayloadVersion = -1;
69 private long replicatedToAllIndex = -1;
71 private final String logName;
73 private final RaftState state;
75 protected AbstractRaftActorBehavior(RaftActorContext context, RaftState state) {
76 this.context = context;
78 this.LOG = context.getLogger();
80 logName = String.format("%s (%s)", context.getId(), state);
84 public RaftState state() {
88 public String logName() {
93 public void setReplicatedToAllIndex(long replicatedToAllIndex) {
94 this.replicatedToAllIndex = replicatedToAllIndex;
98 public long getReplicatedToAllIndex() {
99 return replicatedToAllIndex;
103 * Derived classes should not directly handle AppendEntries messages it
104 * should let the base class handle it first. Once the base class handles
105 * the AppendEntries message and does the common actions that are applicable
106 * in all RaftState's it will delegate the handling of the AppendEntries
107 * message to the derived class to do more state specific handling by calling
110 * @param sender The actor that sent this message
111 * @param appendEntries The AppendEntries message
112 * @return a new behavior if it was changed or the current behavior
114 protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender,
115 AppendEntries appendEntries);
119 * appendEntries first processes the AppendEntries message and then
120 * delegates handling to a specific behavior
123 * @param appendEntries
124 * @return a new behavior if it was changed or the current behavior
126 protected RaftActorBehavior appendEntries(ActorRef sender,
127 AppendEntries appendEntries) {
129 // 1. Reply false if term < currentTerm (§5.1)
130 if (appendEntries.getTerm() < currentTerm()) {
131 if(LOG.isDebugEnabled()) {
132 LOG.debug("{}: Cannot append entries because sender term {} is less than {}",
133 logName(), appendEntries.getTerm(), currentTerm());
137 new AppendEntriesReply(context.getId(), currentTerm(), false,
138 lastIndex(), lastTerm(), context.getPayloadVersion()), actor()
144 return handleAppendEntries(sender, appendEntries);
148 * Derived classes should not directly handle AppendEntriesReply messages it
149 * should let the base class handle it first. Once the base class handles
150 * the AppendEntriesReply message and does the common actions that are
151 * applicable in all RaftState's it will delegate the handling of the
152 * AppendEntriesReply message to the derived class to do more state specific
153 * handling by calling this method
155 * @param sender The actor that sent this message
156 * @param appendEntriesReply The AppendEntriesReply message
157 * @return a new behavior if it was changed or the current behavior
159 protected abstract RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
160 AppendEntriesReply appendEntriesReply);
163 * requestVote handles the RequestVote message. This logic is common
168 * @return a new behavior if it was changed or the current behavior
170 protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) {
172 LOG.debug("{}: In requestVote: {}", logName(), requestVote);
174 boolean grantVote = canGrantVote(requestVote);
177 context.getTermInformation().updateAndPersist(requestVote.getTerm(), requestVote.getCandidateId());
180 RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote);
182 LOG.debug("{}: requestVote returning: {}", logName(), reply);
184 sender.tell(reply, actor());
189 protected boolean canGrantVote(RequestVote requestVote){
190 boolean grantVote = false;
192 // Reply false if term < currentTerm (§5.1)
193 if (requestVote.getTerm() < currentTerm()) {
196 // If votedFor is null or candidateId, and candidate’s log is at
197 // least as up-to-date as receiver’s log, grant vote (§5.2, §5.4)
198 } else if (votedFor() == null || votedFor()
199 .equals(requestVote.getCandidateId())) {
201 boolean candidateLatest = false;
204 // Raft determines which of two logs is more up-to-date
205 // by comparing the index and term of the last entries in the
206 // logs. If the logs have last entries with different terms, then
207 // the log with the later term is more up-to-date. If the logs
208 // end with the same term, then whichever log is longer is
210 if (requestVote.getLastLogTerm() > lastTerm()) {
211 candidateLatest = true;
212 } else if ((requestVote.getLastLogTerm() == lastTerm())
213 && requestVote.getLastLogIndex() >= lastIndex()) {
214 candidateLatest = true;
217 if (candidateLatest) {
225 * Derived classes should not directly handle RequestVoteReply messages it
226 * should let the base class handle it first. Once the base class handles
227 * the RequestVoteReply message and does the common actions that are
228 * applicable in all RaftState's it will delegate the handling of the
229 * RequestVoteReply message to the derived class to do more state specific
230 * handling by calling this method
232 * @param sender The actor that sent this message
233 * @param requestVoteReply The RequestVoteReply message
234 * @return a new behavior if it was changed or the current behavior
236 protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender,
237 RequestVoteReply requestVoteReply);
241 * @return a random election duration
243 protected FiniteDuration electionDuration() {
244 long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
245 return context.getConfigParams().getElectionTimeOutInterval().$plus(
246 new FiniteDuration(variance, TimeUnit.MILLISECONDS));
250 * stop the scheduled election
252 protected void stopElection() {
253 if (electionCancel != null && !electionCancel.isCancelled()) {
254 electionCancel.cancel();
259 * schedule a new election
261 * @param interval the duration after which we should trigger a new election
263 protected void scheduleElection(FiniteDuration interval) {
266 // Schedule an election. When the scheduler triggers an ElectionTimeout
267 // message is sent to itself
269 context.getActorSystem().scheduler().scheduleOnce(interval,
270 context.getActor(), ELECTION_TIMEOUT,
271 context.getActorSystem().dispatcher(), context.getActor());
275 * @return the current term
277 protected long currentTerm() {
278 return context.getTermInformation().getCurrentTerm();
282 * @return the candidate for whom we voted in the current term
284 protected String votedFor() {
285 return context.getTermInformation().getVotedFor();
289 * @return the actor associated with this behavior
291 protected ActorRef actor() {
292 return context.getActor();
297 * @return the term from the last entry in the log
299 protected long lastTerm() {
300 return context.getReplicatedLog().lastTerm();
304 * @return the index from the last entry in the log
306 protected long lastIndex() {
307 return context.getReplicatedLog().lastIndex();
312 * @return the client request tracker for the specified logIndex
314 protected ClientRequestTracker findClientRequestTracker(long logIndex) {
320 * @return the client request tracker for the specified logIndex
322 protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
329 * @return log index from the previous to last entry in the log
331 protected long prevLogIndex(long index){
332 ReplicatedLogEntry prevEntry =
333 context.getReplicatedLog().get(index - 1);
334 if (prevEntry != null) {
335 return prevEntry.getIndex();
341 * @return log term from the previous to last entry in the log
343 protected long prevLogTerm(long index){
344 ReplicatedLogEntry prevEntry =
345 context.getReplicatedLog().get(index - 1);
346 if (prevEntry != null) {
347 return prevEntry.getTerm();
353 * Apply the provided index to the state machine
355 * @param index a log index that is known to be committed
357 protected void applyLogToStateMachine(final long index) {
358 long newLastApplied = context.getLastApplied();
359 // Now maybe we apply to the state machine
360 for (long i = context.getLastApplied() + 1;
361 i < index + 1; i++) {
362 ActorRef clientActor = null;
363 String identifier = null;
364 ClientRequestTracker tracker = removeClientRequestTracker(i);
366 if (tracker != null) {
367 clientActor = tracker.getClientActor();
368 identifier = tracker.getIdentifier();
370 ReplicatedLogEntry replicatedLogEntry =
371 context.getReplicatedLog().get(i);
373 if (replicatedLogEntry != null) {
374 // Send a local message to the local RaftActor (it's derived class to be
375 // specific to apply the log to it's index)
376 actor().tell(new ApplyState(clientActor, identifier,
377 replicatedLogEntry), actor());
380 //if one index is not present in the log, no point in looping
381 // around as the rest wont be present either
383 "{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
384 logName(), i, i, index);
388 if(LOG.isDebugEnabled()) {
389 LOG.debug("{}: Setting last applied to {}", logName(), newLastApplied);
391 context.setLastApplied(newLastApplied);
393 // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
394 // will be used during recovery
395 //in case if the above code throws an error and this message is not sent, it would be fine
396 // as the append entries received later would initiate add this message to the journal
397 actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
400 protected Object fromSerializableMessage(Object serializable){
401 return SerializationUtils.fromSerializable(serializable);
405 public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
406 if (message instanceof AppendEntries) {
407 return appendEntries(sender, (AppendEntries) message);
408 } else if (message instanceof AppendEntriesReply) {
409 return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
410 } else if (message instanceof RequestVote) {
411 return requestVote(sender, (RequestVote) message);
412 } else if (message instanceof RequestVoteReply) {
413 return handleRequestVoteReply(sender, (RequestVoteReply) message);
418 @Override public String getLeaderId() {
423 public short getLeaderPayloadVersion() {
424 return leaderPayloadVersion;
427 public void setLeaderPayloadVersion(short leaderPayloadVersion) {
428 this.leaderPayloadVersion = leaderPayloadVersion;
432 public RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
433 return internalSwitchBehavior(behavior);
436 protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
437 if(context.getRaftPolicy().automaticElectionsEnabled()){
438 return internalSwitchBehavior(newState.createBehavior(context));
443 private RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
444 LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
447 } catch (Exception e) {
448 LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
454 protected int getMajorityVoteCount(int numPeers) {
455 // Votes are required from a majority of the peers including self.
456 // The numMajority field therefore stores a calculated value
457 // of the number of votes required for this candidate to win an
458 // election based on it's known peers.
459 // If a peer was added during normal operation and raft replicas
460 // came to know about them then the new peer would also need to be
461 // taken into consideration when calculating this value.
462 // Here are some examples for what the numMajority would be for n
464 // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
465 // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
466 // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
471 numMajority = (numPeers + self) / 2 + 1;
479 * Performs a snapshot with no capture on the replicated log.
480 * It clears the log from the supplied index or last-applied-1 which ever is minimum.
482 * @param snapshotCapturedIndex
484 protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
485 long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex, this);
487 if(actualIndex != -1){
488 setReplicatedToAllIndex(actualIndex);
492 protected String getId(){
493 return context.getId();
496 public void applyServerConfiguration(ServerConfigurationPayload serverConfig) {
497 Map<String, String> currentPeers = new HashMap<>(context.getPeerAddresses());
498 for(String peerId: serverConfig.getNewServerConfig()) {
499 if(!getId().equals(peerId)) {
500 if(!currentPeers.containsKey(peerId)) {
501 context.addToPeers(peerId, null);
503 currentPeers.remove(peerId);
508 for(String peerIdToRemove: currentPeers.keySet()) {
509 context.removePeer(peerIdToRemove);