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