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