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