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