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