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