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