Merge "Missing capability in 10-rest-connector.xml file"
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.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;
10
11 import akka.actor.ActorRef;
12 import akka.actor.ActorSelection;
13 import akka.event.Logging;
14 import akka.event.LoggingAdapter;
15 import akka.japi.Procedure;
16 import akka.persistence.RecoveryCompleted;
17 import akka.persistence.SaveSnapshotFailure;
18 import akka.persistence.SaveSnapshotSuccess;
19 import akka.persistence.SnapshotOffer;
20 import akka.persistence.SnapshotSelectionCriteria;
21 import akka.persistence.UntypedPersistentActor;
22 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
23 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
24 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
25 import com.google.common.base.Optional;
26 import org.opendaylight.controller.cluster.raft.behaviors.Candidate;
27 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
28 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
29 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
30 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
31 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
32 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
33 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
34 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
35
36 import java.io.Serializable;
37 import java.util.List;
38 import java.util.Map;
39
40 /**
41  * RaftActor encapsulates a state machine that needs to be kept synchronized
42  * in a cluster. It implements the RAFT algorithm as described in the paper
43  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
44  * In Search of an Understandable Consensus Algorithm</a>
45  * <p/>
46  * RaftActor has 3 states and each state has a certain behavior associated
47  * with it. A Raft actor can behave as,
48  * <ul>
49  * <li> A Leader </li>
50  * <li> A Follower (or) </li>
51  * <li> A Candidate </li>
52  * </ul>
53  * <p/>
54  * <p/>
55  * A RaftActor MUST be a Leader in order to accept requests from clients to
56  * change the state of it's encapsulated state machine. Once a RaftActor becomes
57  * a Leader it is also responsible for ensuring that all followers ultimately
58  * have the same log and therefore the same state machine as itself.
59  * <p/>
60  * <p/>
61  * The current behavior of a RaftActor determines how election for leadership
62  * is initiated and how peer RaftActors react to request for votes.
63  * <p/>
64  * <p/>
65  * Each RaftActor also needs to know the current election term. It uses this
66  * information for a couple of things. One is to simply figure out who it
67  * voted for in the last election. Another is to figure out if the message
68  * it received to update it's state is stale.
69  * <p/>
70  * <p/>
71  * The RaftActor uses akka-persistence to store it's replicated log.
72  * Furthermore through it's behaviors a Raft Actor determines
73  * <p/>
74  * <ul>
75  * <li> when a log entry should be persisted </li>
76  * <li> when a log entry should be applied to the state machine (and) </li>
77  * <li> when a snapshot should be saved </li>
78  * </ul>
79  */
80 public abstract class RaftActor extends UntypedPersistentActor {
81     protected final LoggingAdapter LOG =
82         Logging.getLogger(getContext().system(), this);
83
84     /**
85      * The current state determines the current behavior of a RaftActor
86      * A Raft Actor always starts off in the Follower State
87      */
88     private RaftActorBehavior currentBehavior;
89
90     /**
91      * This context should NOT be passed directly to any other actor it is
92      * only to be consumed by the RaftActorBehaviors
93      */
94     private RaftActorContext context;
95
96     /**
97      * The in-memory journal
98      */
99     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
100
101
102     public RaftActor(String id, Map<String, String> peerAddresses) {
103         this(id, peerAddresses, Optional.<ConfigParams>absent());
104     }
105
106     public RaftActor(String id, Map<String, String> peerAddresses,
107          Optional<ConfigParams> configParams) {
108
109         context = new RaftActorContextImpl(this.getSelf(),
110             this.getContext(), id, new ElectionTermImpl(),
111             -1, -1, replicatedLog, peerAddresses,
112             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
113             LOG);
114     }
115
116     @Override public void onReceiveRecover(Object message) {
117         if (message instanceof SnapshotOffer) {
118             LOG.debug("SnapshotOffer called..");
119             SnapshotOffer offer = (SnapshotOffer) message;
120             Snapshot snapshot = (Snapshot) offer.snapshot();
121
122             // Create a replicated log with the snapshot information
123             // The replicated log can be used later on to retrieve this snapshot
124             // when we need to install it on a peer
125             replicatedLog = new ReplicatedLogImpl(snapshot);
126
127             context.setReplicatedLog(replicatedLog);
128
129             LOG.debug("Applied snapshot to replicatedLog. " +
130                 "snapshotIndex={}, snapshotTerm={}, journal-size={}",
131                 replicatedLog.snapshotIndex, replicatedLog.snapshotTerm,
132                 replicatedLog.size());
133
134             // Apply the snapshot to the actors state
135             applySnapshot(snapshot.getState());
136
137         } else if (message instanceof ReplicatedLogEntry) {
138             replicatedLog.append((ReplicatedLogEntry) message);
139         } else if (message instanceof DeleteEntries) {
140             replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
141         } else if (message instanceof UpdateElectionTerm) {
142             context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), ((UpdateElectionTerm) message).getVotedFor());
143         } else if (message instanceof RecoveryCompleted) {
144             LOG.debug(
145                 "RecoveryCompleted - Switching actor to Follower - " +
146                     "Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " +
147                     "journal-size={}",
148                 replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
149                 replicatedLog.snapshotTerm, replicatedLog.size());
150             currentBehavior = switchBehavior(RaftState.Follower);
151             onStateChanged();
152         }
153     }
154
155     @Override public void onReceiveCommand(Object message) {
156         if (message instanceof ApplyState){
157             ApplyState applyState = (ApplyState) message;
158
159             LOG.debug("Applying state for log index {} data {}",
160                 applyState.getReplicatedLogEntry().getIndex(),
161                 applyState.getReplicatedLogEntry().getData());
162
163             applyState(applyState.getClientActor(), applyState.getIdentifier(),
164                 applyState.getReplicatedLogEntry().getData());
165
166         } else if(message instanceof ApplySnapshot ) {
167             applySnapshot(((ApplySnapshot) message).getSnapshot());
168
169         } else if (message instanceof FindLeader) {
170             getSender().tell(
171                 new FindLeaderReply(
172                     context.getPeerAddress(currentBehavior.getLeaderId())),
173                 getSelf()
174             );
175
176         } else if (message instanceof SaveSnapshotSuccess) {
177             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
178
179             // TODO: Not sure if we want to be this aggressive with trimming stuff
180             trimPersistentData(success.metadata().sequenceNr());
181
182         } else if (message instanceof SaveSnapshotFailure) {
183
184             // TODO: Handle failure in saving the snapshot
185
186         } else if (message instanceof FindLeader){
187
188             getSender().tell(new FindLeaderReply(
189                 context.getPeerAddress(currentBehavior.getLeaderId())),
190                 getSelf());
191
192         } else if (message instanceof AddRaftPeer){
193
194             // FIXME : Do not add raft peers like this.
195             // When adding a new Peer we have to ensure that the a majority of
196             // the peers know about the new Peer. Doing it this way may cause
197             // a situation where multiple Leaders may emerge
198             AddRaftPeer arp = (AddRaftPeer)message;
199            context.addToPeers(arp.getName(), arp.getAddress());
200
201         } else if (message instanceof RemoveRaftPeer){
202
203             RemoveRaftPeer rrp = (RemoveRaftPeer)message;
204             context.removePeer(rrp.getName());
205
206         } else {
207
208             RaftState state =
209                 currentBehavior.handleMessage(getSender(), message);
210             RaftActorBehavior oldBehavior = currentBehavior;
211             currentBehavior = switchBehavior(state);
212             if(oldBehavior != currentBehavior){
213                 onStateChanged();
214             }
215         }
216     }
217
218     public java.util.Set<String> getPeers() {
219         return context.getPeerAddresses().keySet();
220     }
221
222     protected String getReplicatedLogState() {
223         return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
224             + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
225             + ", im-mem journal size=" + context.getReplicatedLog().size();
226     }
227
228
229     /**
230      * When a derived RaftActor needs to persist something it must call
231      * persistData.
232      *
233      * @param clientActor
234      * @param identifier
235      * @param data
236      */
237     protected void persistData(ActorRef clientActor, String identifier,
238         Payload data) {
239
240         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
241             context.getReplicatedLog().lastIndex() + 1,
242             context.getTermInformation().getCurrentTerm(), data);
243
244         LOG.debug("Persist data {}", replicatedLogEntry);
245
246         replicatedLog
247             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
248     }
249
250     protected String getId() {
251         return context.getId();
252     }
253
254     /**
255      * Derived actors can call the isLeader method to check if the current
256      * RaftActor is the Leader or not
257      *
258      * @return true it this RaftActor is a Leader false otherwise
259      */
260     protected boolean isLeader() {
261         return context.getId().equals(currentBehavior.getLeaderId());
262     }
263
264     /**
265      * Derived actor can call getLeader if they need a reference to the Leader.
266      * This would be useful for example in forwarding a request to an actor
267      * which is the leader
268      *
269      * @return A reference to the leader if known, null otherwise
270      */
271     protected ActorSelection getLeader(){
272         String leaderId = currentBehavior.getLeaderId();
273         if (leaderId == null) {
274             return null;
275         }
276         String peerAddress = context.getPeerAddress(leaderId);
277         LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = "
278             + peerAddress);
279
280         if(peerAddress == null){
281             return null;
282         }
283         return context.actorSelection(peerAddress);
284     }
285
286     /**
287      *
288      * @return the current leader's id
289      */
290     protected String getLeaderId(){
291         return currentBehavior.getLeaderId();
292     }
293
294     protected RaftState getRaftState() {
295         return currentBehavior.state();
296     }
297
298     protected ReplicatedLogEntry getLastLogEntry() {
299         return replicatedLog.last();
300     }
301
302     protected Long getCurrentTerm(){
303         return context.getTermInformation().getCurrentTerm();
304     }
305
306     protected Long getCommitIndex(){
307         return context.getCommitIndex();
308     }
309
310     protected Long getLastApplied(){
311         return context.getLastApplied();
312     }
313
314     /**
315      * setPeerAddress sets the address of a known peer at a later time.
316      * <p>
317      * This is to account for situations where a we know that a peer
318      * exists but we do not know an address up-front. This may also be used in
319      * situations where a known peer starts off in a different location and we
320      * need to change it's address
321      * <p>
322      * Note that if the peerId does not match the list of peers passed to
323      * this actor during construction an IllegalStateException will be thrown.
324      *
325      * @param peerId
326      * @param peerAddress
327      */
328     protected void setPeerAddress(String peerId, String peerAddress){
329         context.setPeerAddress(peerId, peerAddress);
330     }
331
332
333
334     /**
335      * The applyState method will be called by the RaftActor when some data
336      * needs to be applied to the actor's state
337      *
338      * @param clientActor A reference to the client who sent this message. This
339      *                    is the same reference that was passed to persistData
340      *                    by the derived actor. clientActor may be null when
341      *                    the RaftActor is behaving as a follower or during
342      *                    recovery.
343      * @param identifier  The identifier of the persisted data. This is also
344      *                    the same identifier that was passed to persistData by
345      *                    the derived actor. identifier may be null when
346      *                    the RaftActor is behaving as a follower or during
347      *                    recovery
348      * @param data        A piece of data that was persisted by the persistData call.
349      *                    This should NEVER be null.
350      */
351     protected abstract void applyState(ActorRef clientActor, String identifier,
352         Object data);
353
354     /**
355      * This method will be called by the RaftActor when a snapshot needs to be
356      * created. The derived actor should respond with its current state.
357      * <p/>
358      * During recovery the state that is returned by the derived actor will
359      * be passed back to it by calling the applySnapshot  method
360      *
361      * @return The current state of the actor
362      */
363     protected abstract Object createSnapshot();
364
365     /**
366      * This method will be called by the RaftActor during recovery to
367      * reconstruct the state of the actor.
368      * <p/>
369      * This method may also be called at any other point during normal
370      * operations when the derived actor is out of sync with it's peers
371      * and the only way to bring it in sync is by applying a snapshot
372      *
373      * @param snapshot A snapshot of the state of the actor
374      */
375     protected abstract void applySnapshot(Object snapshot);
376
377     /**
378      * This method will be called by the RaftActor when the state of the
379      * RaftActor changes. The derived actor can then use methods like
380      * isLeader or getLeader to do something useful
381      */
382     protected abstract void onStateChanged();
383
384     private RaftActorBehavior switchBehavior(RaftState state) {
385         if (currentBehavior != null) {
386             if (currentBehavior.state() == state) {
387                 return currentBehavior;
388             }
389             LOG.info("Switching from state " + currentBehavior.state() + " to "
390                 + state);
391
392             try {
393                 currentBehavior.close();
394             } catch (Exception e) {
395                 LOG.error(e,
396                     "Failed to close behavior : " + currentBehavior.state());
397             }
398
399         } else {
400             LOG.info("Switching behavior to " + state);
401         }
402         RaftActorBehavior behavior = null;
403         if (state == RaftState.Candidate) {
404             behavior = new Candidate(context);
405         } else if (state == RaftState.Follower) {
406             behavior = new Follower(context);
407         } else {
408             behavior = new Leader(context);
409         }
410
411
412
413         return behavior;
414     }
415
416     private void trimPersistentData(long sequenceNumber) {
417         // Trim akka snapshots
418         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
419         // For now guessing that it is ANDed.
420         deleteSnapshots(new SnapshotSelectionCriteria(
421             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
422
423         // Trim akka journal
424         deleteMessages(sequenceNumber);
425     }
426
427
428     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
429
430         public ReplicatedLogImpl(Snapshot snapshot) {
431             super(snapshot.getState(),
432                 snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
433                 snapshot.getUnAppliedEntries());
434         }
435
436         public ReplicatedLogImpl() {
437             super();
438         }
439
440         @Override public void removeFromAndPersist(long logEntryIndex) {
441             int adjustedIndex = adjustedIndex(logEntryIndex);
442
443             if (adjustedIndex < 0) {
444                 return;
445             }
446
447             // FIXME: Maybe this should be done after the command is saved
448             journal.subList(adjustedIndex , journal.size()).clear();
449
450             persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
451
452                 @Override public void apply(DeleteEntries param)
453                     throws Exception {
454                     //FIXME : Doing nothing for now
455                 }
456             });
457         }
458
459         @Override public void appendAndPersist(
460             final ReplicatedLogEntry replicatedLogEntry) {
461             appendAndPersist(null, null, replicatedLogEntry);
462         }
463
464         public void appendAndPersist(final ActorRef clientActor,
465             final String identifier,
466             final ReplicatedLogEntry replicatedLogEntry) {
467             context.getLogger().debug(
468                 "Append log entry and persist {} ", replicatedLogEntry);
469             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
470             journal.add(replicatedLogEntry);
471
472             // When persisting events with persist it is guaranteed that the
473             // persistent actor will not receive further commands between the
474             // persist call and the execution(s) of the associated event
475             // handler. This also holds for multiple persist calls in context
476             // of a single command.
477             persist(replicatedLogEntry,
478                 new Procedure<ReplicatedLogEntry>() {
479                     public void apply(ReplicatedLogEntry evt) throws Exception {
480                         // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned.
481                         if (journal.size() > context.getConfigParams().getSnapshotBatchCount()) {
482                             LOG.info("Initiating Snapshot Capture..");
483                             long lastAppliedIndex = -1;
484                             long lastAppliedTerm = -1;
485
486                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
487                             if (lastAppliedEntry != null) {
488                                 lastAppliedIndex = lastAppliedEntry.getIndex();
489                                 lastAppliedTerm = lastAppliedEntry.getTerm();
490                             }
491
492                             LOG.debug("Snapshot Capture logSize: {}", journal.size());
493                             LOG.debug("Snapshot Capture lastApplied:{} ", context.getLastApplied());
494                             LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
495                             LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
496
497                             // create a snapshot object from the state provided and save it
498                             // when snapshot is saved async, SaveSnapshotSuccess is raised.
499                             Snapshot sn = Snapshot.create(createSnapshot(),
500                                 getFrom(context.getLastApplied() + 1),
501                                 lastIndex(), lastTerm(), lastAppliedIndex,
502                                 lastAppliedTerm);
503                             saveSnapshot(sn);
504
505                             LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
506
507                             //be greedy and remove entries from in-mem journal which are in the snapshot
508                             // and update snapshotIndex and snapshotTerm without waiting for the success,
509                             // TODO: damage-recovery to be done on failure
510                             journal.subList(0, (int) (lastAppliedIndex - snapshotIndex)).clear();
511                             snapshotIndex = lastAppliedIndex;
512                             snapshotTerm = lastAppliedTerm;
513
514                             LOG.info("Removed in-memory snapshotted entries, " +
515                                 "adjusted snaphsotIndex:{}" +
516                                 "and term:{}", snapshotIndex, lastAppliedTerm);
517                         }
518                         // Send message for replication
519                         if (clientActor != null) {
520                             currentBehavior.handleMessage(getSelf(),
521                                 new Replicate(clientActor, identifier,
522                                     replicatedLogEntry)
523                             );
524                         }
525                     }
526                 }
527             );
528         }
529
530     }
531
532     private static class DeleteEntries implements Serializable {
533         private final int fromIndex;
534
535
536         public DeleteEntries(int fromIndex) {
537             this.fromIndex = fromIndex;
538         }
539
540         public int getFromIndex() {
541             return fromIndex;
542         }
543     }
544
545
546     private static class Snapshot implements Serializable {
547         private final Object state;
548         private final List<ReplicatedLogEntry> unAppliedEntries;
549         private final long lastIndex;
550         private final long lastTerm;
551         private final long lastAppliedIndex;
552         private final long lastAppliedTerm;
553
554         private Snapshot(Object state,
555             List<ReplicatedLogEntry> unAppliedEntries, long lastIndex,
556             long lastTerm, long lastAppliedIndex, long lastAppliedTerm) {
557             this.state = state;
558             this.unAppliedEntries = unAppliedEntries;
559             this.lastIndex = lastIndex;
560             this.lastTerm = lastTerm;
561             this.lastAppliedIndex = lastAppliedIndex;
562             this.lastAppliedTerm = lastAppliedTerm;
563         }
564
565
566         public static Snapshot create(Object state,
567             List<ReplicatedLogEntry> entries, long lastIndex, long lastTerm,
568             long lastAppliedIndex, long lastAppliedTerm) {
569             return new Snapshot(state, entries, lastIndex, lastTerm,
570                 lastAppliedIndex, lastAppliedTerm);
571         }
572
573         public Object getState() {
574             return state;
575         }
576
577         public List<ReplicatedLogEntry> getUnAppliedEntries() {
578             return unAppliedEntries;
579         }
580
581         public long getLastTerm() {
582             return lastTerm;
583         }
584
585         public long getLastAppliedIndex() {
586             return lastAppliedIndex;
587         }
588
589         public long getLastAppliedTerm() {
590             return lastAppliedTerm;
591         }
592
593         public String getLogMessage() {
594             StringBuilder sb = new StringBuilder();
595             return sb.append("Snapshot={")
596                 .append("lastTerm:" + this.getLastTerm()  + ", ")
597                 .append("LastAppliedIndex:" + this.getLastAppliedIndex()  + ", ")
598                 .append("LastAppliedTerm:" + this.getLastAppliedTerm()  + ", ")
599                 .append("UnAppliedEntries size:" + this.getUnAppliedEntries().size()  + "}")
600                 .toString();
601
602         }
603     }
604
605     private class ElectionTermImpl implements ElectionTerm {
606         /**
607          * Identifier of the actor whose election term information this is
608          */
609         private long currentTerm = 0;
610         private String votedFor = null;
611
612         public long getCurrentTerm() {
613             return currentTerm;
614         }
615
616         public String getVotedFor() {
617             return votedFor;
618         }
619
620         @Override public void update(long currentTerm, String votedFor) {
621             LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
622
623             this.currentTerm = currentTerm;
624             this.votedFor = votedFor;
625         }
626
627         @Override
628         public void updateAndPersist(long currentTerm, String votedFor){
629             update(currentTerm, votedFor);
630             // FIXME : Maybe first persist then update the state
631             persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
632
633                 @Override public void apply(UpdateElectionTerm param)
634                     throws Exception {
635
636                 }
637             });
638         }
639     }
640
641     private static class UpdateElectionTerm implements Serializable {
642         private final long currentTerm;
643         private final String votedFor;
644
645         public UpdateElectionTerm(long currentTerm, String votedFor) {
646             this.currentTerm = currentTerm;
647             this.votedFor = votedFor;
648         }
649
650         public long getCurrentTerm() {
651             return currentTerm;
652         }
653
654         public String getVotedFor() {
655             return votedFor;
656         }
657     }
658
659 }