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