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