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