Merge "Bug 1025: Fixed incorrect revision in sal-remote-augment, which caused log...
[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 org.opendaylight.controller.cluster.raft.ClientRequestTracker;
14 import org.opendaylight.controller.cluster.raft.RaftActorContext;
15 import org.opendaylight.controller.cluster.raft.RaftState;
16 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
17 import org.opendaylight.controller.cluster.raft.SerializationUtils;
18 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
19 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
20 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
21 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
22 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
23 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
25 import scala.concurrent.duration.FiniteDuration;
26
27 import java.util.Random;
28 import java.util.concurrent.TimeUnit;
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     /**
43      * Information about the RaftActor whose behavior this class represents
44      */
45     protected final RaftActorContext context;
46
47     /**
48      *
49      */
50     private Cancellable electionCancel = null;
51
52     /**
53      *
54      */
55     protected String leaderId = null;
56
57
58     protected AbstractRaftActorBehavior(RaftActorContext context) {
59         this.context = context;
60     }
61
62     /**
63      * Derived classes should not directly handle AppendEntries messages it
64      * should let the base class handle it first. Once the base class handles
65      * the AppendEntries message and does the common actions that are applicable
66      * in all RaftState's it will delegate the handling of the AppendEntries
67      * message to the derived class to do more state specific handling by calling
68      * this method
69      *
70      * @param sender         The actor that sent this message
71      * @param appendEntries  The AppendEntries message
72      * @return
73      */
74     protected abstract RaftState handleAppendEntries(ActorRef sender,
75         AppendEntries appendEntries);
76
77
78     /**
79      * appendEntries first processes the AppendEntries message and then
80      * delegates handling to a specific behavior
81      *
82      * @param sender
83      * @param appendEntries
84      * @return
85      */
86     protected RaftState appendEntries(ActorRef sender,
87         AppendEntries appendEntries) {
88
89         // 1. Reply false if term < currentTerm (§5.1)
90         if (appendEntries.getTerm() < currentTerm()) {
91             context.getLogger().debug(
92                 "Cannot append entries because sender term " + appendEntries
93                     .getTerm() + " is less than " + currentTerm());
94             sender.tell(
95                 new AppendEntriesReply(context.getId(), currentTerm(), false,
96                     lastIndex(), lastTerm()), actor()
97             );
98             return state();
99         }
100
101
102         return handleAppendEntries(sender, appendEntries);
103     }
104
105     /**
106      * Derived classes should not directly handle AppendEntriesReply messages it
107      * should let the base class handle it first. Once the base class handles
108      * the AppendEntriesReply message and does the common actions that are
109      * applicable in all RaftState's it will delegate the handling of the
110      * AppendEntriesReply message to the derived class to do more state specific
111      * handling by calling this method
112      *
113      * @param sender             The actor that sent this message
114      * @param appendEntriesReply The AppendEntriesReply message
115      * @return
116      */
117     protected abstract RaftState handleAppendEntriesReply(ActorRef sender,
118         AppendEntriesReply appendEntriesReply);
119
120     /**
121      * requestVote handles the RequestVote message. This logic is common
122      * for all behaviors
123      *
124      * @param sender
125      * @param requestVote
126      * @return
127      */
128     protected RaftState requestVote(ActorRef sender,
129         RequestVote requestVote) {
130
131
132         context.getLogger().debug(requestVote.toString());
133
134         boolean grantVote = false;
135
136         //  Reply false if term < currentTerm (§5.1)
137         if (requestVote.getTerm() < currentTerm()) {
138             grantVote = false;
139
140             // If votedFor is null or candidateId, and candidate’s log is at
141             // least as up-to-date as receiver’s log, grant vote (§5.2, §5.4)
142         } else if (votedFor() == null || votedFor()
143             .equals(requestVote.getCandidateId())) {
144
145             boolean candidateLatest = false;
146
147             // From §5.4.1
148             // Raft determines which of two logs is more up-to-date
149             // by comparing the index and term of the last entries in the
150             // logs. If the logs have last entries with different terms, then
151             // the log with the later term is more up-to-date. If the logs
152             // end with the same term, then whichever log is longer is
153             // more up-to-date.
154             if (requestVote.getLastLogTerm() > lastTerm()) {
155                 candidateLatest = true;
156             } else if ((requestVote.getLastLogTerm() == lastTerm())
157                 && requestVote.getLastLogIndex() >= lastIndex()) {
158                 candidateLatest = true;
159             }
160
161             if (candidateLatest) {
162                 grantVote = true;
163                 context.getTermInformation().updateAndPersist(requestVote.getTerm(),
164                     requestVote.getCandidateId());
165             }
166         }
167
168         sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor());
169
170         return state();
171     }
172
173     /**
174      * Derived classes should not directly handle RequestVoteReply messages it
175      * should let the base class handle it first. Once the base class handles
176      * the RequestVoteReply message and does the common actions that are
177      * applicable in all RaftState's it will delegate the handling of the
178      * RequestVoteReply message to the derived class to do more state specific
179      * handling by calling this method
180      *
181      * @param sender           The actor that sent this message
182      * @param requestVoteReply The RequestVoteReply message
183      * @return
184      */
185     protected abstract RaftState handleRequestVoteReply(ActorRef sender,
186         RequestVoteReply requestVoteReply);
187
188     /**
189      * Creates a random election duration
190      *
191      * @return
192      */
193     protected FiniteDuration electionDuration() {
194         long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
195         return context.getConfigParams().getElectionTimeOutInterval().$plus(
196             new FiniteDuration(variance, TimeUnit.MILLISECONDS));
197     }
198
199     /**
200      * stop the scheduled election
201      */
202     protected void stopElection() {
203         if (electionCancel != null && !electionCancel.isCancelled()) {
204             electionCancel.cancel();
205         }
206     }
207
208     /**
209      * schedule a new election
210      *
211      * @param interval
212      */
213     protected void scheduleElection(FiniteDuration interval) {
214         stopElection();
215
216         // Schedule an election. When the scheduler triggers an ElectionTimeout
217         // message is sent to itself
218         electionCancel =
219             context.getActorSystem().scheduler().scheduleOnce(interval,
220                 context.getActor(), new ElectionTimeout(),
221                 context.getActorSystem().dispatcher(), context.getActor());
222     }
223
224     /**
225      * Get the current term
226      * @return
227      */
228     protected long currentTerm() {
229         return context.getTermInformation().getCurrentTerm();
230     }
231
232     /**
233      * Get the candidate for whom we voted in the current term
234      * @return
235      */
236     protected String votedFor() {
237         return context.getTermInformation().getVotedFor();
238     }
239
240     /**
241      * Get the actor associated with this behavior
242      * @return
243      */
244     protected ActorRef actor() {
245         return context.getActor();
246     }
247
248     /**
249      * Get the term from the last entry in the log
250      *
251      * @return
252      */
253     protected long lastTerm() {
254         return context.getReplicatedLog().lastTerm();
255     }
256
257     /**
258      * Get the index from the last entry in the log
259      *
260      * @return
261      */
262     protected long lastIndex() {
263         return context.getReplicatedLog().lastIndex();
264     }
265
266     /**
267      * Find the client request tracker for a specific logIndex
268      *
269      * @param logIndex
270      * @return
271      */
272     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
273         return null;
274     }
275
276     /**
277      * Find the client request tracker for a specific logIndex
278      *
279      * @param logIndex
280      * @return
281      */
282     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
283         return null;
284     }
285
286
287     /**
288      * Find the log index from the previous to last entry in the log
289      *
290      * @return
291      */
292     protected long prevLogIndex(long index){
293         ReplicatedLogEntry prevEntry =
294             context.getReplicatedLog().get(index - 1);
295         if (prevEntry != null) {
296             return prevEntry.getIndex();
297         }
298         return -1;
299     }
300
301     /**
302      * Find the log term from the previous to last entry in the log
303      * @return
304      */
305     protected long prevLogTerm(long index){
306         ReplicatedLogEntry prevEntry =
307             context.getReplicatedLog().get(index - 1);
308         if (prevEntry != null) {
309             return prevEntry.getTerm();
310         }
311         return -1;
312     }
313
314     /**
315      * Apply the provided index to the state machine
316      *
317      * @param index a log index that is known to be committed
318      */
319     protected void applyLogToStateMachine(final long index) {
320         long newLastApplied = context.getLastApplied();
321         // Now maybe we apply to the state machine
322         for (long i = context.getLastApplied() + 1;
323              i < index + 1; i++) {
324             ActorRef clientActor = null;
325             String identifier = null;
326             ClientRequestTracker tracker = removeClientRequestTracker(i);
327
328             if (tracker != null) {
329                 clientActor = tracker.getClientActor();
330                 identifier = tracker.getIdentifier();
331             }
332             ReplicatedLogEntry replicatedLogEntry =
333                 context.getReplicatedLog().get(i);
334
335             if (replicatedLogEntry != null) {
336                 // Send a local message to the local RaftActor (it's derived class to be
337                 // specific to apply the log to it's index)
338                 actor().tell(new ApplyState(clientActor, identifier,
339                     replicatedLogEntry), actor());
340                 newLastApplied = i;
341             } else {
342                 //if one index is not present in the log, no point in looping
343                 // around as the rest wont be present either
344                 context.getLogger().warning(
345                     "Missing index {} from log. Cannot apply state. Ignoring {} to {}", i, i, index );
346                 break;
347             }
348         }
349         context.getLogger().debug("Setting last applied to {}", newLastApplied);
350         context.setLastApplied(newLastApplied);
351
352         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
353         // will be used during recovery
354         //in case if the above code throws an error and this message is not sent, it would be fine
355         // as the  append entries received later would initiate add this message to the journal
356         actor().tell(new ApplyLogEntries((int) context.getLastApplied()), actor());
357     }
358
359     protected Object fromSerializableMessage(Object serializable){
360         return SerializationUtils.fromSerializable(serializable);
361     }
362
363     @Override
364     public RaftState handleMessage(ActorRef sender, Object message) {
365         if (message instanceof AppendEntries) {
366             return appendEntries(sender, (AppendEntries) message);
367         } else if (message instanceof AppendEntriesReply) {
368             return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
369         } else if (message instanceof RequestVote) {
370             return requestVote(sender, (RequestVote) message);
371         } else if (message instanceof RequestVoteReply) {
372             return handleRequestVoteReply(sender, (RequestVoteReply) message);
373         }
374         return state();
375     }
376
377     @Override public String getLeaderId() {
378         return leaderId;
379     }
380 }