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