e8c1b09772f172157c177c747b1bc2f7c5b51430
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / behaviors / AbstractRaftActorBehavior.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
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
7  */
8
9 package org.opendaylight.controller.cluster.raft.behaviors;
10
11 import akka.actor.ActorRef;
12 import akka.actor.Cancellable;
13 import com.google.common.base.Preconditions;
14 import java.util.Random;
15 import java.util.concurrent.TimeUnit;
16 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
17 import org.opendaylight.controller.cluster.raft.RaftActorContext;
18 import org.opendaylight.controller.cluster.raft.RaftState;
19 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
20 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
21 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
22 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
23 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
25 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
26 import org.opendaylight.controller.cluster.raft.persisted.ApplyJournalEntries;
27 import org.slf4j.Logger;
28 import scala.concurrent.duration.FiniteDuration;
29
30 /**
31  * Abstract class that provides common code for a RaftActor behavior.
32  */
33 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
34     /**
35      * Information about the RaftActor whose behavior this class represents.
36      */
37     protected final RaftActorContext context;
38
39     /**
40      * Used for message logging.
41      */
42     protected final Logger log;
43
44     /**
45      * Prepended to log messages to provide appropriate context.
46      */
47     private final String logName;
48
49     /**
50      * The RaftState corresponding to his behavior.
51      */
52     private final RaftState state;
53
54     /**
55      * Used to cancel a scheduled election.
56      */
57     private Cancellable electionCancel = null;
58
59     /**
60      * The index of the last log entry that has been replicated to all raft peers.
61      */
62     private long replicatedToAllIndex = -1;
63
64     AbstractRaftActorBehavior(final RaftActorContext context, final RaftState state) {
65         this.context = Preconditions.checkNotNull(context);
66         this.state = Preconditions.checkNotNull(state);
67         this.log = context.getLogger();
68
69         logName = String.format("%s (%s)", context.getId(), state);
70     }
71
72     public static RaftActorBehavior createBehavior(final RaftActorContext context, final RaftState state) {
73         switch (state) {
74             case Candidate:
75                 return new Candidate(context);
76             case Follower:
77                 return new Follower(context);
78             case IsolatedLeader:
79                 return new IsolatedLeader(context);
80             case Leader:
81                 return new Leader(context);
82             case PreLeader:
83                 return new PreLeader(context);
84             default:
85                 throw new IllegalArgumentException("Unhandled state " + state);
86         }
87     }
88
89     @Override
90     public final RaftState state() {
91         return state;
92     }
93
94     protected final String logName() {
95         return logName;
96     }
97
98     @Override
99     public void setReplicatedToAllIndex(long replicatedToAllIndex) {
100         this.replicatedToAllIndex = replicatedToAllIndex;
101     }
102
103     @Override
104     public long getReplicatedToAllIndex() {
105         return replicatedToAllIndex;
106     }
107
108     /**
109      * Derived classes should not directly handle AppendEntries messages it
110      * should let the base class handle it first. Once the base class handles
111      * the AppendEntries message and does the common actions that are applicable
112      * in all RaftState's it will delegate the handling of the AppendEntries
113      * message to the derived class to do more state specific handling by calling
114      * this method
115      *
116      * @param sender         The actor that sent this message
117      * @param appendEntries  The AppendEntries message
118      * @return a new behavior if it was changed or the current behavior
119      */
120     protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender,
121         AppendEntries appendEntries);
122
123     /**
124      * Handles the common logic for the AppendEntries message and delegates handling to the derived class.
125      *
126      * @param sender the ActorRef that sent the message
127      * @param appendEntries the message
128      * @return a new behavior if it was changed or the current behavior
129      */
130     protected RaftActorBehavior appendEntries(ActorRef sender, AppendEntries appendEntries) {
131
132         // 1. Reply false if term < currentTerm (ยง5.1)
133         if (appendEntries.getTerm() < currentTerm()) {
134             log.info("{}: Cannot append entries because sender's term {} is less than {}", logName(),
135                     appendEntries.getTerm(), currentTerm());
136
137             sender.tell(new AppendEntriesReply(context.getId(), currentTerm(), false, lastIndex(), lastTerm(),
138                     context.getPayloadVersion()), actor());
139             return this;
140         }
141
142
143         return handleAppendEntries(sender, appendEntries);
144     }
145
146     /**
147      * Derived classes should not directly handle AppendEntriesReply messages it
148      * should let the base class handle it first. Once the base class handles
149      * the AppendEntriesReply message and does the common actions that are
150      * applicable in all RaftState's it will delegate the handling of the
151      * AppendEntriesReply message to the derived class to do more state specific
152      * handling by calling this method
153      *
154      * @param sender             The actor that sent this message
155      * @param appendEntriesReply The AppendEntriesReply message
156      * @return a new behavior if it was changed or the current behavior
157      */
158     protected abstract RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
159         AppendEntriesReply appendEntriesReply);
160
161     /**
162      * Handles the logic for the RequestVote message that is common for all behaviors.
163      *
164      * @param sender the ActorRef that sent the message
165      * @param requestVote the message
166      * @return a new behavior if it was changed or the current behavior
167      */
168     protected RaftActorBehavior requestVote(ActorRef sender, RequestVote requestVote) {
169
170         log.debug("{}: In requestVote:  {} - currentTerm: {}, votedFor: {}, lastIndex: {}, lastTerm: {}", logName(),
171                 requestVote, currentTerm(), votedFor(), lastIndex(), lastTerm());
172
173         boolean grantVote = canGrantVote(requestVote);
174
175         if (grantVote) {
176             context.getTermInformation().updateAndPersist(requestVote.getTerm(), requestVote.getCandidateId());
177         }
178
179         RequestVoteReply reply = new RequestVoteReply(currentTerm(), grantVote);
180
181         log.debug("{}: requestVote returning: {}", logName(), reply);
182
183         sender.tell(reply, actor());
184
185         return this;
186     }
187
188     protected boolean canGrantVote(RequestVote requestVote) {
189         boolean grantVote = false;
190
191         //  Reply false if term < currentTerm (ยง5.1)
192         if (requestVote.getTerm() < currentTerm()) {
193             grantVote = false;
194
195             // If votedFor is null or candidateId, and candidateโ€™s log is at
196             // least as up-to-date as receiverโ€™s log, grant vote (ยง5.2, ยง5.4)
197         } else if (votedFor() == null || votedFor()
198                 .equals(requestVote.getCandidateId())) {
199
200             boolean candidateLatest = false;
201
202             // From ยง5.4.1
203             // Raft determines which of two logs is more up-to-date
204             // by comparing the index and term of the last entries in the
205             // logs. If the logs have last entries with different terms, then
206             // the log with the later term is more up-to-date. If the logs
207             // end with the same term, then whichever log is longer is
208             // more up-to-date.
209             if (requestVote.getLastLogTerm() > lastTerm()) {
210                 candidateLatest = true;
211             } else if (requestVote.getLastLogTerm() == lastTerm()
212                     && requestVote.getLastLogIndex() >= lastIndex()) {
213                 candidateLatest = true;
214             }
215
216             if (candidateLatest) {
217                 grantVote = true;
218             }
219         }
220         return grantVote;
221     }
222
223     /**
224      * Derived classes should not directly handle RequestVoteReply messages it
225      * should let the base class handle it first. Once the base class handles
226      * the RequestVoteReply message and does the common actions that are
227      * applicable in all RaftState's it will delegate the handling of the
228      * RequestVoteReply message to the derived class to do more state specific
229      * handling by calling this method
230      *
231      * @param sender           The actor that sent this message
232      * @param requestVoteReply The RequestVoteReply message
233      * @return a new behavior if it was changed or the current behavior
234      */
235     protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender,
236         RequestVoteReply requestVoteReply);
237
238     /**
239      * Returns a duration for election with an additional variance for randomness.
240      *
241      * @return a random election duration
242      */
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));
247     }
248
249     /**
250      * Stops the currently scheduled election.
251      */
252     protected void stopElection() {
253         if (electionCancel != null && !electionCancel.isCancelled()) {
254             electionCancel.cancel();
255         }
256     }
257
258     protected boolean canStartElection() {
259         return context.getRaftPolicy().automaticElectionsEnabled() && context.isVotingMember();
260     }
261
262     /**
263      * Schedule a new election.
264      *
265      * @param interval the duration after which we should trigger a new election
266      */
267     protected void scheduleElection(FiniteDuration interval) {
268         stopElection();
269
270         // Schedule an election. When the scheduler triggers an ElectionTimeout message is sent to itself
271         electionCancel = context.getActorSystem().scheduler().scheduleOnce(interval, context.getActor(),
272                 ElectionTimeout.INSTANCE, context.getActorSystem().dispatcher(), context.getActor());
273     }
274
275     /**
276      * Returns the current election term.
277      *
278      * @return the current term
279      */
280     protected long currentTerm() {
281         return context.getTermInformation().getCurrentTerm();
282     }
283
284     /**
285      * Returns the id of the candidate that this server voted for in current term.
286      *
287      * @return the candidate for whom we voted in the current term
288      */
289     protected String votedFor() {
290         return context.getTermInformation().getVotedFor();
291     }
292
293     /**
294      * Returns the actor associated with this behavior.
295      *
296      * @return the actor
297      */
298     protected ActorRef actor() {
299         return context.getActor();
300     }
301
302     /**
303      * Returns the term of the last entry in the log.
304      *
305      * @return the term
306      */
307     protected long lastTerm() {
308         return context.getReplicatedLog().lastTerm();
309     }
310
311     /**
312      * Returns the index of the last entry in the log.
313      *
314      * @return the index
315      */
316     protected long lastIndex() {
317         return context.getReplicatedLog().lastIndex();
318     }
319
320     /**
321      * Removes and returns the ClientRequestTracker for the specified log index.
322      * @param logIndex the log index
323      * @return the ClientRequestTracker or null if none available
324      */
325     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
326         return null;
327     }
328
329     /**
330      * Returns the actual index of the entry in replicated log for the given index or -1 if not found.
331      *
332      * @return the log entry index or -1 if not found
333      */
334     protected long getLogEntryIndex(long index) {
335         if (index == context.getReplicatedLog().getSnapshotIndex()) {
336             return context.getReplicatedLog().getSnapshotIndex();
337         }
338
339         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
340         if (entry != null) {
341             return entry.getIndex();
342         }
343
344         return -1;
345     }
346
347     /**
348      * Returns the actual term of the entry in replicated log for the given index or -1 if not found.
349      *
350      * @return the log entry term or -1 if not found
351      */
352     protected long getLogEntryTerm(long index) {
353         if (index == context.getReplicatedLog().getSnapshotIndex()) {
354             return context.getReplicatedLog().getSnapshotTerm();
355         }
356
357         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
358         if (entry != null) {
359             return entry.getTerm();
360         }
361
362         return -1;
363     }
364
365     /**
366      * Applies the log entries up to the specified index that is known to be committed to the state machine.
367      *
368      * @param index the log index
369      */
370     protected void applyLogToStateMachine(final long index) {
371         // Now maybe we apply to the state machine
372         for (long i = context.getLastApplied() + 1; i < index + 1; i++) {
373
374             ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(i);
375             if (replicatedLogEntry != null) {
376                 // Send a local message to the local RaftActor (it's derived class to be
377                 // specific to apply the log to it's index)
378
379                 final ApplyState applyState;
380                 final ClientRequestTracker tracker = removeClientRequestTracker(i);
381                 if (tracker != null) {
382                     applyState = new ApplyState(tracker.getClientActor(), tracker.getIdentifier(), replicatedLogEntry);
383                 } else {
384                     applyState = new ApplyState(null, null, replicatedLogEntry);
385                 }
386
387                 log.debug("{}: Setting last applied to {}", logName(), i);
388
389                 context.setLastApplied(i);
390                 context.getApplyStateConsumer().accept(applyState);
391             } else {
392                 //if one index is not present in the log, no point in looping
393                 // around as the rest wont be present either
394                 log.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
395                         logName(), i, i, index);
396                 break;
397             }
398         }
399
400         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
401         // will be used during recovery
402         //in case if the above code throws an error and this message is not sent, it would be fine
403         // as the  append entries received later would initiate add this message to the journal
404         actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
405     }
406
407     @Override
408     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
409         if (message instanceof AppendEntries) {
410             return appendEntries(sender, (AppendEntries) message);
411         } else if (message instanceof AppendEntriesReply) {
412             return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
413         } else if (message instanceof RequestVote) {
414             return requestVote(sender, (RequestVote) message);
415         } else if (message instanceof RequestVoteReply) {
416             return handleRequestVoteReply(sender, (RequestVoteReply) message);
417         } else {
418             return null;
419         }
420     }
421
422     @Override
423     public RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
424         return internalSwitchBehavior(behavior);
425     }
426
427     protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
428         return internalSwitchBehavior(createBehavior(context, newState));
429     }
430
431     @SuppressWarnings("checkstyle:IllegalCatch")
432     protected RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
433         if (!context.getRaftPolicy().automaticElectionsEnabled()) {
434             return this;
435         }
436
437         log.info("{} :- Switching from behavior {} to {}, election term: {}", logName(), this.state(),
438                 newBehavior.state(), context.getTermInformation().getCurrentTerm());
439         try {
440             close();
441         } catch (RuntimeException e) {
442             log.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
443         }
444         return newBehavior;
445     }
446
447
448     protected int getMajorityVoteCount(int numPeers) {
449         // Votes are required from a majority of the peers including self.
450         // The numMajority field therefore stores a calculated value
451         // of the number of votes required for this candidate to win an
452         // election based on it's known peers.
453         // If a peer was added during normal operation and raft replicas
454         // came to know about them then the new peer would also need to be
455         // taken into consideration when calculating this value.
456         // Here are some examples for what the numMajority would be for n
457         // peers
458         // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
459         // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
460         // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
461
462         int numMajority = 0;
463         if (numPeers > 0) {
464             int self = 1;
465             numMajority = (numPeers + self) / 2 + 1;
466         }
467         return numMajority;
468
469     }
470
471
472     /**
473      * Performs a snapshot with no capture on the replicated log. It clears the log from the supplied index or
474      * lastApplied-1 which ever is minimum.
475      *
476      * @param snapshotCapturedIndex the index from which to clear
477      */
478     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
479         long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
480
481         if (actualIndex != -1) {
482             setReplicatedToAllIndex(actualIndex);
483         }
484     }
485
486     protected String getId() {
487         return context.getId();
488     }
489 }