Merge "Bug-2692:Real snapshots should use the replicatedToAllIndex for clearing in...
[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 java.util.Random;
14 import java.util.concurrent.TimeUnit;
15 import org.opendaylight.controller.cluster.raft.ClientRequestTracker;
16 import org.opendaylight.controller.cluster.raft.RaftActorContext;
17 import org.opendaylight.controller.cluster.raft.ReplicatedLogEntry;
18 import org.opendaylight.controller.cluster.raft.SerializationUtils;
19 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
20 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
21 import org.opendaylight.controller.cluster.raft.base.messages.ElectionTimeout;
22 import org.opendaylight.controller.cluster.raft.messages.AppendEntries;
23 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
24 import org.opendaylight.controller.cluster.raft.messages.RequestVote;
25 import org.opendaylight.controller.cluster.raft.messages.RequestVoteReply;
26 import org.slf4j.Logger;
27 import scala.concurrent.duration.FiniteDuration;
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     protected final Logger LOG;
50
51     /**
52      *
53      */
54     private Cancellable electionCancel = null;
55
56     /**
57      *
58      */
59     protected String leaderId = null;
60
61     private long replicatedToAllIndex = -1;
62
63     protected AbstractRaftActorBehavior(RaftActorContext context) {
64         this.context = context;
65         this.LOG = context.getLogger();
66     }
67
68     @Override
69     public void setReplicatedToAllIndex(long replicatedToAllIndex) {
70         this.replicatedToAllIndex = replicatedToAllIndex;
71     }
72
73     @Override
74     public long getReplicatedToAllIndex() {
75         return replicatedToAllIndex;
76     }
77
78     /**
79      * Derived classes should not directly handle AppendEntries messages it
80      * should let the base class handle it first. Once the base class handles
81      * the AppendEntries message and does the common actions that are applicable
82      * in all RaftState's it will delegate the handling of the AppendEntries
83      * message to the derived class to do more state specific handling by calling
84      * this method
85      *
86      * @param sender         The actor that sent this message
87      * @param appendEntries  The AppendEntries message
88      * @return
89      */
90     protected abstract RaftActorBehavior handleAppendEntries(ActorRef sender,
91         AppendEntries appendEntries);
92
93
94     /**
95      * appendEntries first processes the AppendEntries message and then
96      * delegates handling to a specific behavior
97      *
98      * @param sender
99      * @param appendEntries
100      * @return
101      */
102     protected RaftActorBehavior appendEntries(ActorRef sender,
103         AppendEntries appendEntries) {
104
105         // 1. Reply false if term < currentTerm (§5.1)
106         if (appendEntries.getTerm() < currentTerm()) {
107             if(LOG.isDebugEnabled()) {
108                 LOG.debug("{}: Cannot append entries because sender term {} is less than {}",
109                         context.getId(), appendEntries.getTerm(), currentTerm());
110             }
111
112             sender.tell(
113                 new AppendEntriesReply(context.getId(), currentTerm(), false,
114                     lastIndex(), lastTerm()), actor()
115             );
116             return this;
117         }
118
119
120         return handleAppendEntries(sender, appendEntries);
121     }
122
123     /**
124      * Derived classes should not directly handle AppendEntriesReply messages it
125      * should let the base class handle it first. Once the base class handles
126      * the AppendEntriesReply message and does the common actions that are
127      * applicable in all RaftState's it will delegate the handling of the
128      * AppendEntriesReply message to the derived class to do more state specific
129      * handling by calling this method
130      *
131      * @param sender             The actor that sent this message
132      * @param appendEntriesReply The AppendEntriesReply message
133      * @return
134      */
135     protected abstract RaftActorBehavior handleAppendEntriesReply(ActorRef sender,
136         AppendEntriesReply appendEntriesReply);
137
138     /**
139      * requestVote handles the RequestVote message. This logic is common
140      * for all behaviors
141      *
142      * @param sender
143      * @param requestVote
144      * @return
145      */
146     protected RaftActorBehavior requestVote(ActorRef sender,
147         RequestVote requestVote) {
148
149         if(LOG.isDebugEnabled()) {
150             LOG.debug("{}: Received {}", context.getId(), requestVote);
151         }
152
153         boolean grantVote = false;
154
155         //  Reply false if term < currentTerm (§5.1)
156         if (requestVote.getTerm() < currentTerm()) {
157             grantVote = false;
158
159             // If votedFor is null or candidateId, and candidate’s log is at
160             // least as up-to-date as receiver’s log, grant vote (§5.2, §5.4)
161         } else if (votedFor() == null || votedFor()
162             .equals(requestVote.getCandidateId())) {
163
164             boolean candidateLatest = false;
165
166             // From §5.4.1
167             // Raft determines which of two logs is more up-to-date
168             // by comparing the index and term of the last entries in the
169             // logs. If the logs have last entries with different terms, then
170             // the log with the later term is more up-to-date. If the logs
171             // end with the same term, then whichever log is longer is
172             // more up-to-date.
173             if (requestVote.getLastLogTerm() > lastTerm()) {
174                 candidateLatest = true;
175             } else if ((requestVote.getLastLogTerm() == lastTerm())
176                 && requestVote.getLastLogIndex() >= lastIndex()) {
177                 candidateLatest = true;
178             }
179
180             if (candidateLatest) {
181                 grantVote = true;
182                 context.getTermInformation().updateAndPersist(requestVote.getTerm(),
183                     requestVote.getCandidateId());
184             }
185         }
186
187         sender.tell(new RequestVoteReply(currentTerm(), grantVote), actor());
188
189         return this;
190     }
191
192     /**
193      * Derived classes should not directly handle RequestVoteReply messages it
194      * should let the base class handle it first. Once the base class handles
195      * the RequestVoteReply message and does the common actions that are
196      * applicable in all RaftState's it will delegate the handling of the
197      * RequestVoteReply message to the derived class to do more state specific
198      * handling by calling this method
199      *
200      * @param sender           The actor that sent this message
201      * @param requestVoteReply The RequestVoteReply message
202      * @return
203      */
204     protected abstract RaftActorBehavior handleRequestVoteReply(ActorRef sender,
205         RequestVoteReply requestVoteReply);
206
207     /**
208      * Creates a random election duration
209      *
210      * @return
211      */
212     protected FiniteDuration electionDuration() {
213         long variance = new Random().nextInt(context.getConfigParams().getElectionTimeVariance());
214         return context.getConfigParams().getElectionTimeOutInterval().$plus(
215                 new FiniteDuration(variance, TimeUnit.MILLISECONDS));
216     }
217
218     /**
219      * stop the scheduled election
220      */
221     protected void stopElection() {
222         if (electionCancel != null && !electionCancel.isCancelled()) {
223             electionCancel.cancel();
224         }
225     }
226
227     /**
228      * schedule a new election
229      *
230      * @param interval
231      */
232     protected void scheduleElection(FiniteDuration interval) {
233         stopElection();
234
235         // Schedule an election. When the scheduler triggers an ElectionTimeout
236         // message is sent to itself
237         electionCancel =
238             context.getActorSystem().scheduler().scheduleOnce(interval,
239                 context.getActor(), new ElectionTimeout(),
240                 context.getActorSystem().dispatcher(), context.getActor());
241     }
242
243     /**
244      * Get the current term
245      * @return
246      */
247     protected long currentTerm() {
248         return context.getTermInformation().getCurrentTerm();
249     }
250
251     /**
252      * Get the candidate for whom we voted in the current term
253      * @return
254      */
255     protected String votedFor() {
256         return context.getTermInformation().getVotedFor();
257     }
258
259     /**
260      * Get the actor associated with this behavior
261      * @return
262      */
263     protected ActorRef actor() {
264         return context.getActor();
265     }
266
267     /**
268      * Get the term from the last entry in the log
269      *
270      * @return
271      */
272     protected long lastTerm() {
273         return context.getReplicatedLog().lastTerm();
274     }
275
276     /**
277      * Get the index from the last entry in the log
278      *
279      * @return
280      */
281     protected long lastIndex() {
282         return context.getReplicatedLog().lastIndex();
283     }
284
285     /**
286      * Find the client request tracker for a specific logIndex
287      *
288      * @param logIndex
289      * @return
290      */
291     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
292         return null;
293     }
294
295     /**
296      * Find the client request tracker for a specific logIndex
297      *
298      * @param logIndex
299      * @return
300      */
301     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
302         return null;
303     }
304
305
306     /**
307      * Find the log index from the previous to last entry in the log
308      *
309      * @return
310      */
311     protected long prevLogIndex(long index){
312         ReplicatedLogEntry prevEntry =
313             context.getReplicatedLog().get(index - 1);
314         if (prevEntry != null) {
315             return prevEntry.getIndex();
316         }
317         return -1;
318     }
319
320     /**
321      * Find the log term from the previous to last entry in the log
322      * @return
323      */
324     protected long prevLogTerm(long index){
325         ReplicatedLogEntry prevEntry =
326             context.getReplicatedLog().get(index - 1);
327         if (prevEntry != null) {
328             return prevEntry.getTerm();
329         }
330         return -1;
331     }
332
333     /**
334      * Apply the provided index to the state machine
335      *
336      * @param index a log index that is known to be committed
337      */
338     protected void applyLogToStateMachine(final long index) {
339         long newLastApplied = context.getLastApplied();
340         // Now maybe we apply to the state machine
341         for (long i = context.getLastApplied() + 1;
342              i < index + 1; i++) {
343             ActorRef clientActor = null;
344             String identifier = null;
345             ClientRequestTracker tracker = removeClientRequestTracker(i);
346
347             if (tracker != null) {
348                 clientActor = tracker.getClientActor();
349                 identifier = tracker.getIdentifier();
350             }
351             ReplicatedLogEntry replicatedLogEntry =
352                 context.getReplicatedLog().get(i);
353
354             if (replicatedLogEntry != null) {
355                 // Send a local message to the local RaftActor (it's derived class to be
356                 // specific to apply the log to it's index)
357                 actor().tell(new ApplyState(clientActor, identifier,
358                     replicatedLogEntry), actor());
359                 newLastApplied = i;
360             } else {
361                 //if one index is not present in the log, no point in looping
362                 // around as the rest wont be present either
363                 LOG.warn(
364                         "{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
365                         context.getId(), i, i, index);
366                 break;
367             }
368         }
369         if(LOG.isDebugEnabled()) {
370             LOG.debug("{}: Setting last applied to {}", context.getId(), newLastApplied);
371         }
372         context.setLastApplied(newLastApplied);
373
374         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
375         // will be used during recovery
376         //in case if the above code throws an error and this message is not sent, it would be fine
377         // as the  append entries received later would initiate add this message to the journal
378         actor().tell(new ApplyLogEntries((int) context.getLastApplied()), actor());
379     }
380
381     protected Object fromSerializableMessage(Object serializable){
382         return SerializationUtils.fromSerializable(serializable);
383     }
384
385     @Override
386     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
387         if (message instanceof AppendEntries) {
388             return appendEntries(sender, (AppendEntries) message);
389         } else if (message instanceof AppendEntriesReply) {
390             return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
391         } else if (message instanceof RequestVote) {
392             return requestVote(sender, (RequestVote) message);
393         } else if (message instanceof RequestVoteReply) {
394             return handleRequestVoteReply(sender, (RequestVoteReply) message);
395         }
396         return this;
397     }
398
399     @Override public String getLeaderId() {
400         return leaderId;
401     }
402
403     protected RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
404         LOG.info("{} :- Switching from behavior {} to {}", context.getId(), this.state(), behavior.state());
405         try {
406             close();
407         } catch (Exception e) {
408             LOG.error("{}: Failed to close behavior : {}", context.getId(), this.state(), e);
409         }
410
411         return behavior;
412     }
413
414     protected int getMajorityVoteCount(int numPeers) {
415         // Votes are required from a majority of the peers including self.
416         // The numMajority field therefore stores a calculated value
417         // of the number of votes required for this candidate to win an
418         // election based on it's known peers.
419         // If a peer was added during normal operation and raft replicas
420         // came to know about them then the new peer would also need to be
421         // taken into consideration when calculating this value.
422         // Here are some examples for what the numMajority would be for n
423         // peers
424         // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
425         // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
426         // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
427
428         int numMajority = 0;
429         if (numPeers > 0) {
430             int self = 1;
431             numMajority = (numPeers + self) / 2 + 1;
432         }
433         return numMajority;
434
435     }
436
437
438     /**
439      * Performs a snapshot with no capture on the replicated log.
440      * It clears the log from the supplied index or last-applied-1 which ever is minimum.
441      *
442      * @param snapshotCapturedIndex
443      */
444     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
445         //  we would want to keep the lastApplied as its used while capturing snapshots
446         long lastApplied = context.getLastApplied();
447         long tempMin = Math.min(snapshotCapturedIndex, (lastApplied > -1 ? lastApplied - 1 : -1));
448
449         if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin))  {
450             //use the term of the temp-min, since we check for isPresent, entry will not be null
451             ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
452             context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
453             context.getReplicatedLog().snapshotCommit();
454             setReplicatedToAllIndex(tempMin);
455         }
456     }
457
458 }