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