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