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