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