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