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