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