fd2fbd332c7a58bab6f60b01e37b2193ad98c3e7
[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.Random;
19 import java.util.Set;
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         this.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()) {
216                 candidateLatest = true;
217             } else if (requestVote.getLastLogTerm() == lastTerm()
218                     && requestVote.getLastLogIndex() >= lastIndex()) {
219                 candidateLatest = true;
220             }
221
222             if (candidateLatest) {
223                 grantVote = true;
224             }
225         }
226         return grantVote;
227     }
228
229     /**
230      * Derived classes should not directly handle RequestVoteReply messages it
231      * should let the base class handle it first. Once the base class handles
232      * the RequestVoteReply message and does the common actions that are
233      * applicable in all RaftState's it will delegate the handling of the
234      * RequestVoteReply message to the derived class to do more state specific
235      * handling by calling this method
236      *
237      * @param sender           The actor that sent this message
238      * @param requestVoteReply The RequestVoteReply message
239      * @return a new behavior if it was changed or the current behavior
240      */
241     protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender,
242         RequestVoteReply requestVoteReply);
243
244     /**
245      * Returns a duration for election with an additional variance for randomness.
246      *
247      * @return a random election duration
248      */
249     protected FiniteDuration electionDuration() {
250         long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
251         return context.getConfigParams().getElectionTimeOutInterval().$plus(
252                 new FiniteDuration(variance, TimeUnit.MILLISECONDS));
253     }
254
255     /**
256      * Stops the currently scheduled election.
257      */
258     protected void stopElection() {
259         if (electionCancel != null && !electionCancel.isCancelled()) {
260             electionCancel.cancel();
261         }
262     }
263
264     protected boolean canStartElection() {
265         return context.getRaftPolicy().automaticElectionsEnabled() && context.isVotingMember();
266     }
267
268     /**
269      * Schedule a new election.
270      *
271      * @param interval the duration after which we should trigger a new election
272      */
273     protected void scheduleElection(final FiniteDuration interval) {
274         stopElection();
275
276         // Schedule an election. When the scheduler triggers an ElectionTimeout message is sent to itself
277         electionCancel = context.getActorSystem().scheduler().scheduleOnce(interval, context.getActor(),
278                 ElectionTimeout.INSTANCE, context.getActorSystem().dispatcher(), context.getActor());
279     }
280
281     /**
282      * Returns the current election term.
283      *
284      * @return the current term
285      */
286     protected long currentTerm() {
287         return context.getTermInformation().getCurrentTerm();
288     }
289
290     /**
291      * Returns the id of the candidate that this server voted for in current term.
292      *
293      * @return the candidate for whom we voted in the current term
294      */
295     protected String votedFor() {
296         return context.getTermInformation().getVotedFor();
297     }
298
299     /**
300      * Returns the actor associated with this behavior.
301      *
302      * @return the actor
303      */
304     protected ActorRef actor() {
305         return context.getActor();
306     }
307
308     /**
309      * Returns the term of the last entry in the log.
310      *
311      * @return the term
312      */
313     protected long lastTerm() {
314         return context.getReplicatedLog().lastTerm();
315     }
316
317     /**
318      * Returns the index of the last entry in the log.
319      *
320      * @return the index
321      */
322     protected long lastIndex() {
323         return context.getReplicatedLog().lastIndex();
324     }
325
326     /**
327      * Returns the actual index of the entry in replicated log for the given index or -1 if not found.
328      *
329      * @return the log entry index or -1 if not found
330      */
331     protected long getLogEntryIndex(final long index) {
332         if (index == context.getReplicatedLog().getSnapshotIndex()) {
333             return context.getReplicatedLog().getSnapshotIndex();
334         }
335
336         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
337         if (entry != null) {
338             return entry.getIndex();
339         }
340
341         return -1;
342     }
343
344     /**
345      * Returns the actual term of the entry in the replicated log for the given index or -1 if not found.
346      *
347      * @return the log entry term or -1 if not found
348      */
349     protected long getLogEntryTerm(final long index) {
350         if (index == context.getReplicatedLog().getSnapshotIndex()) {
351             return context.getReplicatedLog().getSnapshotTerm();
352         }
353
354         ReplicatedLogEntry entry = context.getReplicatedLog().get(index);
355         if (entry != null) {
356             return entry.getTerm();
357         }
358
359         return -1;
360     }
361
362     /**
363      * Returns the actual term of the entry in the replicated log for the given index or, if not present, returns the
364      * snapshot term if the given index is in the snapshot or -1 otherwise.
365      *
366      * @return the term or -1 otherwise
367      */
368     protected long getLogEntryOrSnapshotTerm(final long index) {
369         if (context.getReplicatedLog().isInSnapshot(index)) {
370             return context.getReplicatedLog().getSnapshotTerm();
371         }
372
373         return getLogEntryTerm(index);
374     }
375
376     /**
377      * Applies the log entries up to the specified index that is known to be committed to the state machine.
378      *
379      * @param index the log index
380      */
381     protected void applyLogToStateMachine(final long index) {
382         // Now maybe we apply to the state machine
383         for (long i = context.getLastApplied() + 1; i < index + 1; i++) {
384
385             ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(i);
386             if (replicatedLogEntry != null) {
387                 // Send a local message to the local RaftActor (it's derived class to be
388                 // specific to apply the log to it's index)
389
390                 final ApplyState applyState = getApplyStateFor(replicatedLogEntry);
391
392                 log.debug("{}: Setting last applied to {}", logName(), i);
393
394                 context.setLastApplied(i);
395                 context.getApplyStateConsumer().accept(applyState);
396             } else {
397                 //if one index is not present in the log, no point in looping
398                 // around as the rest wont be present either
399                 log.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
400                         logName(), i, i, index);
401                 break;
402             }
403         }
404
405         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
406         // will be used during recovery
407         //in case if the above code throws an error and this message is not sent, it would be fine
408         // as the  append entries received later would initiate add this message to the journal
409         actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
410     }
411
412     /**
413      * Create an ApplyState message for a particular log entry so we can determine how to apply this entry.
414      *
415      * @param entry the log entry
416      * @return ApplyState for this entry
417      */
418     abstract ApplyState getApplyStateFor(ReplicatedLogEntry entry);
419
420     @Override
421     public RaftActorBehavior handleMessage(final ActorRef sender, final Object message) {
422         if (message instanceof AppendEntries) {
423             return appendEntries(sender, (AppendEntries) message);
424         } else if (message instanceof AppendEntriesReply) {
425             return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
426         } else if (message instanceof RequestVote) {
427             return requestVote(sender, (RequestVote) message);
428         } else if (message instanceof RequestVoteReply) {
429             return handleRequestVoteReply(sender, (RequestVoteReply) message);
430         } else {
431             return null;
432         }
433     }
434
435     @Override
436     public RaftActorBehavior switchBehavior(final RaftActorBehavior behavior) {
437         return internalSwitchBehavior(behavior);
438     }
439
440     protected RaftActorBehavior internalSwitchBehavior(final RaftState newState) {
441         return internalSwitchBehavior(createBehavior(context, newState));
442     }
443
444     @SuppressWarnings("checkstyle:IllegalCatch")
445     protected RaftActorBehavior internalSwitchBehavior(final RaftActorBehavior newBehavior) {
446         if (!context.getRaftPolicy().automaticElectionsEnabled()) {
447             return this;
448         }
449
450         log.info("{} :- Switching from behavior {} to {}, election term: {}", logName(), this.state(),
451                 newBehavior.state(), context.getTermInformation().getCurrentTerm());
452         try {
453             close();
454         } catch (RuntimeException e) {
455             log.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
456         }
457         return newBehavior;
458     }
459
460
461     protected int getMajorityVoteCount(final int numPeers) {
462         // Votes are required from a majority of the peers including self.
463         // The numMajority field therefore stores a calculated value
464         // of the number of votes required for this candidate to win an
465         // election based on it's known peers.
466         // If a peer was added during normal operation and raft replicas
467         // came to know about them then the new peer would also need to be
468         // taken into consideration when calculating this value.
469         // Here are some examples for what the numMajority would be for n
470         // peers
471         // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
472         // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
473         // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
474
475         int numMajority = 0;
476         if (numPeers > 0) {
477             int self = 1;
478             numMajority = (numPeers + self) / 2 + 1;
479         }
480         return numMajority;
481
482     }
483
484
485     /**
486      * Performs a snapshot with no capture on the replicated log. It clears the log from the supplied index or
487      * lastApplied-1 which ever is minimum.
488      *
489      * @param snapshotCapturedIndex the index from which to clear
490      */
491     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
492         long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
493
494         if (actualIndex != -1) {
495             setReplicatedToAllIndex(actualIndex);
496         }
497     }
498
499     protected String getId() {
500         return context.getId();
501     }
502
503     // Check whether we should update the term. In case of half-connected nodes, we want to ignore RequestVote
504     // messages, as the candidate is not able to receive our response.
505     protected boolean shouldUpdateTerm(final RaftRPC rpc) {
506         if (!(rpc instanceof RequestVote)) {
507             return true;
508         }
509
510         final RequestVote requestVote = (RequestVote) rpc;
511         log.debug("{}: Found higher term in RequestVote rpc, verifying whether it's safe to update term.", logName());
512         final Optional<Cluster> maybeCluster = context.getCluster();
513         if (!maybeCluster.isPresent()) {
514             return true;
515         }
516
517         final Cluster cluster = maybeCluster.get();
518
519         final Set<Member> unreachable = cluster.state().getUnreachable();
520         log.debug("{}: Cluster state: {}", logName(), unreachable);
521
522         for (Member member : unreachable) {
523             for (String role : member.getRoles()) {
524                 if (requestVote.getCandidateId().startsWith(role)) {
525                     log.debug("{}: Unreachable member: {}, matches candidateId in: {}, not updating term", logName(),
526                         member, requestVote);
527                     return false;
528                 }
529             }
530         }
531
532         log.debug("{}: Candidate in requestVote:{} with higher term appears reachable, updating term.", logName(),
533             requestVote);
534         return true;
535     }
536 }