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