BUG-5626: use Identifier instead of String
[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.opendaylight.yangtools.concepts.Identifier;
29 import org.slf4j.Logger;
30 import scala.concurrent.duration.FiniteDuration;
31
32 /**
33  * Abstract class that represents the behavior of a RaftActor
34  * <p/>
35  * All Servers:
36  * <ul>
37  * <li> If commitIndex > lastApplied: increment lastApplied, apply
38  * log[lastApplied] to state machine (§5.3)
39  * <li> If RPC request or response contains term T > currentTerm:
40  * set currentTerm = T, convert to follower (§5.1)
41  */
42 public abstract class AbstractRaftActorBehavior implements RaftActorBehavior {
43     /**
44      * Information about the RaftActor whose behavior this class represents
45      */
46     protected final RaftActorContext context;
47
48     /**
49      *
50      */
51     protected final Logger LOG;
52
53     /**
54      *
55      */
56     private Cancellable electionCancel = null;
57
58     private long replicatedToAllIndex = -1;
59
60     private final String logName;
61
62     private final RaftState state;
63
64     AbstractRaftActorBehavior(final RaftActorContext context, final RaftState state) {
65         this.context = Preconditions.checkNotNull(context);
66         this.state = Preconditions.checkNotNull(state);
67         this.LOG = context.getLogger();
68
69         logName = String.format("%s (%s)", context.getId(), state);
70     }
71
72     public static RaftActorBehavior createBehavior(final RaftActorContext context, final RaftState state) {
73         switch (state) {
74             case Candidate:
75                 return new Candidate(context);
76             case Follower:
77                 return new Follower(context);
78             case IsolatedLeader:
79                 return new IsolatedLeader(context);
80             case Leader:
81                 return new Leader(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         if(canStartElection()) {
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     /**
282      * @return the current term
283      */
284     protected long currentTerm() {
285         return context.getTermInformation().getCurrentTerm();
286     }
287
288     /**
289      * @return the candidate for whom we voted in the current term
290      */
291     protected String votedFor() {
292         return context.getTermInformation().getVotedFor();
293     }
294
295     /**
296      * @return the actor associated with this behavior
297      */
298     protected ActorRef actor() {
299         return context.getActor();
300     }
301
302     /**
303      *
304      * @return the term from the last entry in the log
305      */
306     protected long lastTerm() {
307         return context.getReplicatedLog().lastTerm();
308     }
309
310     /**
311      * @return the index from the last entry in the log
312      */
313     protected long lastIndex() {
314         return context.getReplicatedLog().lastIndex();
315     }
316
317     /**
318      * @param logIndex
319      * @return the client request tracker for the specified logIndex
320      */
321     protected ClientRequestTracker findClientRequestTracker(long logIndex) {
322         return null;
323     }
324
325     /**
326      * @param logIndex
327      * @return the client request tracker for the specified logIndex
328      */
329     protected ClientRequestTracker removeClientRequestTracker(long logIndex) {
330         return null;
331     }
332
333
334     /**
335      *
336      * @return log index from the previous to last entry in the log
337      */
338     protected long prevLogIndex(long index){
339         ReplicatedLogEntry prevEntry =
340             context.getReplicatedLog().get(index - 1);
341         if (prevEntry != null) {
342             return prevEntry.getIndex();
343         }
344         return -1;
345     }
346
347     /**
348      * @return log term from the previous to last entry in the log
349      */
350     protected long prevLogTerm(long index){
351         ReplicatedLogEntry prevEntry =
352             context.getReplicatedLog().get(index - 1);
353         if (prevEntry != null) {
354             return prevEntry.getTerm();
355         }
356         return -1;
357     }
358
359     /**
360      * Apply the provided index to the state machine
361      *
362      * @param index a log index that is known to be committed
363      */
364     protected void applyLogToStateMachine(final long index) {
365         long newLastApplied = context.getLastApplied();
366         // Now maybe we apply to the state machine
367         for (long i = context.getLastApplied() + 1; i < index + 1; i++) {
368             final ActorRef clientActor;
369             final Identifier identifier;
370             final ClientRequestTracker tracker = removeClientRequestTracker(i);
371             if (tracker != null) {
372                 clientActor = tracker.getClientActor();
373                 identifier = tracker.getIdentifier();
374             } else {
375                 clientActor = null;
376                 identifier = null;
377             }
378
379             ReplicatedLogEntry replicatedLogEntry = context.getReplicatedLog().get(i);
380             if (replicatedLogEntry != null) {
381                 // Send a local message to the local RaftActor (it's derived class to be
382                 // specific to apply the log to it's index)
383                 actor().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), actor());
384                 newLastApplied = i;
385             } else {
386                 //if one index is not present in the log, no point in looping
387                 // around as the rest wont be present either
388                 LOG.warn("{}: Missing index {} from log. Cannot apply state. Ignoring {} to {}",
389                         logName(), i, i, index);
390                 break;
391             }
392         }
393         if(LOG.isDebugEnabled()) {
394             LOG.debug("{}: Setting last applied to {}", logName(), newLastApplied);
395         }
396         context.setLastApplied(newLastApplied);
397
398         // send a message to persist a ApplyLogEntries marker message into akka's persistent journal
399         // will be used during recovery
400         //in case if the above code throws an error and this message is not sent, it would be fine
401         // as the  append entries received later would initiate add this message to the journal
402         actor().tell(new ApplyJournalEntries(context.getLastApplied()), actor());
403     }
404
405     protected Object fromSerializableMessage(Object serializable){
406         return SerializationUtils.fromSerializable(serializable);
407     }
408
409     @Override
410     public RaftActorBehavior handleMessage(ActorRef sender, Object message) {
411         if (message instanceof AppendEntries) {
412             return appendEntries(sender, (AppendEntries) message);
413         } else if (message instanceof AppendEntriesReply) {
414             return handleAppendEntriesReply(sender, (AppendEntriesReply) message);
415         } else if (message instanceof RequestVote) {
416             return requestVote(sender, (RequestVote) message);
417         } else if (message instanceof RequestVoteReply) {
418             return handleRequestVoteReply(sender, (RequestVoteReply) message);
419         } else {
420             return null;
421         }
422     }
423
424     @Override
425     public RaftActorBehavior switchBehavior(RaftActorBehavior behavior) {
426         return internalSwitchBehavior(behavior);
427     }
428
429     protected RaftActorBehavior internalSwitchBehavior(RaftState newState) {
430         if(context.getRaftPolicy().automaticElectionsEnabled()){
431             return internalSwitchBehavior(createBehavior(context, newState));
432         }
433         return this;
434     }
435
436     private RaftActorBehavior internalSwitchBehavior(RaftActorBehavior newBehavior) {
437         LOG.info("{} :- Switching from behavior {} to {}", logName(), this.state(), newBehavior.state());
438         try {
439             close();
440         } catch (Exception e) {
441             LOG.error("{}: Failed to close behavior : {}", logName(), this.state(), e);
442         }
443         return newBehavior;
444     }
445
446
447     protected int getMajorityVoteCount(int numPeers) {
448         // Votes are required from a majority of the peers including self.
449         // The numMajority field therefore stores a calculated value
450         // of the number of votes required for this candidate to win an
451         // election based on it's known peers.
452         // If a peer was added during normal operation and raft replicas
453         // came to know about them then the new peer would also need to be
454         // taken into consideration when calculating this value.
455         // Here are some examples for what the numMajority would be for n
456         // peers
457         // 0 peers = 1 numMajority -: (0 + 1) / 2 + 1 = 1
458         // 2 peers = 2 numMajority -: (2 + 1) / 2 + 1 = 2
459         // 4 peers = 3 numMajority -: (4 + 1) / 2 + 1 = 3
460
461         int numMajority = 0;
462         if (numPeers > 0) {
463             int self = 1;
464             numMajority = (numPeers + self) / 2 + 1;
465         }
466         return numMajority;
467
468     }
469
470
471     /**
472      * Performs a snapshot with no capture on the replicated log.
473      * It clears the log from the supplied index or last-applied-1 which ever is minimum.
474      *
475      * @param snapshotCapturedIndex
476      */
477     protected void performSnapshotWithoutCapture(final long snapshotCapturedIndex) {
478         long actualIndex = context.getSnapshotManager().trimLog(snapshotCapturedIndex);
479
480         if(actualIndex != -1){
481             setReplicatedToAllIndex(actualIndex);
482         }
483     }
484
485     protected String getId(){
486         return context.getId();
487     }
488 }