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