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