added feature topology manager shell
[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.behaviors.Candidate;
23 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
24 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
25 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
26 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
27 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
28 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
29 import org.opendaylight.controller.cluster.raft.internal.messages.ApplySnapshot;
30 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
31 import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState;
32 import org.opendaylight.controller.cluster.raft.internal.messages.Replicate;
33 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
34
35 import java.io.Serializable;
36 import java.util.ArrayList;
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         context = new RaftActorContextImpl(this.getSelf(),
104             this.getContext(),
105             id, new ElectionTermImpl(),
106             -1, -1, replicatedLog, peerAddresses, LOG);
107     }
108
109     @Override public void onReceiveRecover(Object message) {
110         if (message instanceof SnapshotOffer) {
111             SnapshotOffer offer = (SnapshotOffer) message;
112             Snapshot snapshot = (Snapshot) offer.snapshot();
113
114             // Create a replicated log with the snapshot information
115             // The replicated log can be used later on to retrieve this snapshot
116             // when we need to install it on a peer
117             replicatedLog = new ReplicatedLogImpl(snapshot);
118
119             // Apply the snapshot to the actors state
120             applySnapshot(snapshot.getState());
121
122         } else if (message instanceof ReplicatedLogEntry) {
123             replicatedLog.append((ReplicatedLogEntry) message);
124         } else if (message instanceof DeleteEntries) {
125             replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
126         } else if (message instanceof UpdateElectionTerm) {
127             context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), ((UpdateElectionTerm) message).getVotedFor());
128         } else if (message instanceof RecoveryCompleted) {
129             LOG.debug(
130                 "Last index in log : " + replicatedLog.lastIndex());
131             currentBehavior = switchBehavior(RaftState.Follower);
132         }
133     }
134
135     @Override public void onReceiveCommand(Object message) {
136         if (message instanceof ApplyState){
137             ApplyState applyState = (ApplyState) message;
138
139             LOG.debug("Applying state for log index {} data {}",
140                 applyState.getReplicatedLogEntry().getIndex(),
141                 applyState.getReplicatedLogEntry().getData());
142
143             applyState(applyState.getClientActor(), applyState.getIdentifier(),
144                 applyState.getReplicatedLogEntry().getData());
145
146         } else if(message instanceof ApplySnapshot ) {
147             applySnapshot(((ApplySnapshot) message).getSnapshot());
148
149         } else if (message instanceof FindLeader) {
150             getSender().tell(
151                 new FindLeaderReply(
152                     context.getPeerAddress(currentBehavior.getLeaderId())),
153                 getSelf()
154             );
155
156         } else if (message instanceof SaveSnapshotSuccess) {
157             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
158
159             // TODO: Not sure if we want to be this aggressive with trimming stuff
160             trimPersistentData(success.metadata().sequenceNr());
161
162         } else if (message instanceof SaveSnapshotFailure) {
163
164             // TODO: Handle failure in saving the snapshot
165
166         } else if (message instanceof FindLeader){
167
168             getSender().tell(new FindLeaderReply(
169                 context.getPeerAddress(currentBehavior.getLeaderId())),
170                 getSelf());
171
172         } else if (message instanceof AddRaftPeer){
173
174             // FIXME : Do not add raft peers like this.
175             // When adding a new Peer we have to ensure that the a majority of
176             // the peers know about the new Peer. Doing it this way may cause
177             // a situation where multiple Leaders may emerge
178             AddRaftPeer arp = (AddRaftPeer)message;
179            context.addToPeers(arp.getName(), arp.getAddress());
180
181         } else if (message instanceof RemoveRaftPeer){
182
183             RemoveRaftPeer rrp = (RemoveRaftPeer)message;
184             context.removePeer(rrp.getName());
185
186         } else {
187
188             RaftState state =
189                 currentBehavior.handleMessage(getSender(), message);
190             currentBehavior = switchBehavior(state);
191         }
192     }
193
194
195
196     /**
197      * When a derived RaftActor needs to persist something it must call
198      * persistData.
199      *
200      * @param clientActor
201      * @param identifier
202      * @param data
203      */
204     protected void persistData(ActorRef clientActor, String identifier,
205         Payload data) {
206
207         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
208             context.getReplicatedLog().lastIndex() + 1,
209             context.getTermInformation().getCurrentTerm(), data);
210
211         LOG.debug("Persist data {}", replicatedLogEntry);
212
213         replicatedLog
214             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
215     }
216
217     protected String getId() {
218         return context.getId();
219     }
220
221     /**
222      * Derived actors can call the isLeader method to check if the current
223      * RaftActor is the Leader or not
224      *
225      * @return true it this RaftActor is a Leader false otherwise
226      */
227     protected boolean isLeader() {
228         return context.getId().equals(currentBehavior.getLeaderId());
229     }
230
231     /**
232      * Derived actor can call getLeader if they need a reference to the Leader.
233      * This would be useful for example in forwarding a request to an actor
234      * which is the leader
235      *
236      * @return A reference to the leader if known, null otherwise
237      */
238     protected ActorSelection getLeader(){
239         String leaderId = currentBehavior.getLeaderId();
240         if (leaderId == null) {
241             return null;
242         }
243         String peerAddress = context.getPeerAddress(leaderId);
244         LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = "
245             + peerAddress);
246         return context.actorSelection(peerAddress);
247     }
248
249     protected RaftState getRaftState() {
250         return currentBehavior.state();
251     }
252
253
254
255     /**
256      * The applyState method will be called by the RaftActor when some data
257      * needs to be applied to the actor's state
258      *
259      * @param clientActor A reference to the client who sent this message. This
260      *                    is the same reference that was passed to persistData
261      *                    by the derived actor. clientActor may be null when
262      *                    the RaftActor is behaving as a follower or during
263      *                    recovery.
264      * @param identifier  The identifier of the persisted data. This is also
265      *                    the same identifier that was passed to persistData by
266      *                    the derived actor. identifier may be null when
267      *                    the RaftActor is behaving as a follower or during
268      *                    recovery
269      * @param data        A piece of data that was persisted by the persistData call.
270      *                    This should NEVER be null.
271      */
272     protected abstract void applyState(ActorRef clientActor, String identifier,
273         Object data);
274
275     /**
276      * This method will be called by the RaftActor when a snapshot needs to be
277      * created. The derived actor should respond with its current state.
278      * <p/>
279      * During recovery the state that is returned by the derived actor will
280      * be passed back to it by calling the applySnapshot  method
281      *
282      * @return The current state of the actor
283      */
284     protected abstract Object createSnapshot();
285
286     /**
287      * This method will be called by the RaftActor during recovery to
288      * reconstruct the state of the actor.
289      * <p/>
290      * This method may also be called at any other point during normal
291      * operations when the derived actor is out of sync with it's peers
292      * and the only way to bring it in sync is by applying a snapshot
293      *
294      * @param snapshot A snapshot of the state of the actor
295      */
296     protected abstract void applySnapshot(Object snapshot);
297
298     private RaftActorBehavior switchBehavior(RaftState state) {
299         if (currentBehavior != null) {
300             if (currentBehavior.state() == state) {
301                 return currentBehavior;
302             }
303             LOG.info("Switching from state " + currentBehavior.state() + " to "
304                 + state);
305
306             try {
307                 currentBehavior.close();
308             } catch (Exception e) {
309                 LOG.error(e,
310                     "Failed to close behavior : " + currentBehavior.state());
311             }
312
313         } else {
314             LOG.info("Switching behavior to " + state);
315         }
316         RaftActorBehavior behavior = null;
317         if (state == RaftState.Candidate) {
318             behavior = new Candidate(context);
319         } else if (state == RaftState.Follower) {
320             behavior = new Follower(context);
321         } else {
322             behavior = new Leader(context);
323         }
324         return behavior;
325     }
326
327     private void trimPersistentData(long sequenceNumber) {
328         // Trim snapshots
329         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
330         // For now guessing that it is ANDed.
331         deleteSnapshots(new SnapshotSelectionCriteria(
332             sequenceNumber - 100000, 43200000));
333
334         // Trim journal
335         deleteMessages(sequenceNumber);
336     }
337
338
339     private class ReplicatedLogImpl implements ReplicatedLog {
340         private final List<ReplicatedLogEntry> journal;
341         private final Object snapshot;
342         private long snapshotIndex = -1;
343         private long snapshotTerm = -1;
344
345         public ReplicatedLogImpl(Snapshot snapshot) {
346             this.snapshot = snapshot.getState();
347             this.snapshotIndex = snapshot.getLastAppliedIndex();
348             this.snapshotTerm = snapshot.getLastAppliedTerm();
349
350             this.journal = new ArrayList<>(snapshot.getUnAppliedEntries());
351         }
352
353         public ReplicatedLogImpl() {
354             this.snapshot = null;
355             this.journal = new ArrayList<>();
356         }
357
358         @Override public ReplicatedLogEntry get(long index) {
359             int adjustedIndex = adjustedIndex(index);
360
361             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
362                 return null;
363             }
364
365             return journal.get(adjustedIndex);
366         }
367
368         @Override public ReplicatedLogEntry last() {
369             if (journal.size() == 0) {
370                 return null;
371             }
372             return get(journal.size() - 1);
373         }
374
375         @Override public long lastIndex() {
376             if (journal.size() == 0) {
377                 return -1;
378             }
379
380             return last().getIndex();
381         }
382
383         @Override public long lastTerm() {
384             if (journal.size() == 0) {
385                 return -1;
386             }
387
388             return last().getTerm();
389         }
390
391
392         @Override public void removeFrom(long index) {
393             int adjustedIndex = adjustedIndex(index);
394
395             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
396                 return;
397             }
398
399             journal.subList(adjustedIndex , journal.size()).clear();
400         }
401
402
403         @Override public void removeFromAndPersist(long index) {
404             int adjustedIndex = adjustedIndex(index);
405
406             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
407                 return;
408             }
409
410             // FIXME: Maybe this should be done after the command is saved
411             journal.subList(adjustedIndex , journal.size()).clear();
412
413             persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
414
415                 @Override public void apply(DeleteEntries param)
416                     throws Exception {
417                     //FIXME : Doing nothing for now
418                 }
419             });
420
421
422         }
423
424         @Override public void append(
425             final ReplicatedLogEntry replicatedLogEntry) {
426             journal.add(replicatedLogEntry);
427         }
428
429         @Override public List<ReplicatedLogEntry> getFrom(long index) {
430             int adjustedIndex = adjustedIndex(index);
431
432             List<ReplicatedLogEntry> entries = new ArrayList<>(100);
433             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
434                 return entries;
435             }
436
437
438             for (int i = adjustedIndex;
439                  i < journal.size(); i++) {
440                 entries.add(journal.get(i));
441             }
442             return entries;
443         }
444
445         @Override public void appendAndPersist(
446             final ReplicatedLogEntry replicatedLogEntry) {
447             appendAndPersist(null, null, replicatedLogEntry);
448         }
449
450         public void appendAndPersist(final ActorRef clientActor,
451             final String identifier,
452             final ReplicatedLogEntry replicatedLogEntry) {
453             context.getLogger().debug(
454                 "Append log entry and persist " + replicatedLogEntry);
455             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
456             journal.add(replicatedLogEntry);
457
458             // When persisting events with persist it is guaranteed that the
459             // persistent actor will not receive further commands between the
460             // persist call and the execution(s) of the associated event
461             // handler. This also holds for multiple persist calls in context
462             // of a single command.
463             persist(replicatedLogEntry,
464                 new Procedure<ReplicatedLogEntry>() {
465                     public void apply(ReplicatedLogEntry evt) throws Exception {
466                         // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned.
467                         if (size() > 100000) {
468                             ReplicatedLogEntry lastAppliedEntry =
469                                 get(context.getLastApplied());
470                             long lastAppliedIndex = -1;
471                             long lastAppliedTerm = -1;
472                             if (lastAppliedEntry != null) {
473                                 lastAppliedIndex = lastAppliedEntry.getIndex();
474                                 lastAppliedTerm = lastAppliedEntry.getTerm();
475                             }
476
477                             saveSnapshot(Snapshot.create(createSnapshot(),
478                                 getFrom(context.getLastApplied() + 1),
479                                 lastIndex(), lastTerm(), lastAppliedIndex,
480                                 lastAppliedTerm));
481                         }
482                         // Send message for replication
483                         if (clientActor != null) {
484                             currentBehavior.handleMessage(getSelf(),
485                                 new Replicate(clientActor, identifier,
486                                     replicatedLogEntry)
487                             );
488                         }
489                     }
490                 }
491             );
492         }
493
494         @Override public long size() {
495             return journal.size() + snapshotIndex + 1;
496         }
497
498         @Override public boolean isPresent(long index) {
499             int adjustedIndex = adjustedIndex(index);
500
501             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
502                 return false;
503             }
504             return true;
505         }
506
507         @Override public boolean isInSnapshot(long index) {
508             return index <= snapshotIndex;
509         }
510
511         @Override public Object getSnapshot() {
512             return snapshot;
513         }
514
515         @Override public long getSnapshotIndex() {
516             return snapshotIndex;
517         }
518
519         @Override public long getSnapshotTerm() {
520             return snapshotTerm;
521         }
522
523         private int adjustedIndex(long index) {
524             if(snapshotIndex < 0){
525                 return (int) index;
526             }
527             return (int) (index - snapshotIndex);
528         }
529     }
530
531
532
533
534     private static class DeleteEntries implements Serializable {
535         private final int fromIndex;
536
537
538         public DeleteEntries(int fromIndex) {
539             this.fromIndex = fromIndex;
540         }
541
542         public int getFromIndex() {
543             return fromIndex;
544         }
545     }
546
547
548     private static class Snapshot implements Serializable {
549         private final Object state;
550         private final List<ReplicatedLogEntry> unAppliedEntries;
551         private final long lastIndex;
552         private final long lastTerm;
553         private final long lastAppliedIndex;
554         private final long lastAppliedTerm;
555
556         private Snapshot(Object state,
557             List<ReplicatedLogEntry> unAppliedEntries, long lastIndex,
558             long lastTerm, long lastAppliedIndex, long lastAppliedTerm) {
559             this.state = state;
560             this.unAppliedEntries = unAppliedEntries;
561             this.lastIndex = lastIndex;
562             this.lastTerm = lastTerm;
563             this.lastAppliedIndex = lastAppliedIndex;
564             this.lastAppliedTerm = lastAppliedTerm;
565         }
566
567
568         public static Snapshot create(Object state,
569             List<ReplicatedLogEntry> entries, long lastIndex, long lastTerm,
570             long lastAppliedIndex, long lastAppliedTerm) {
571             return new Snapshot(state, entries, lastIndex, lastTerm,
572                 lastAppliedIndex, lastAppliedTerm);
573         }
574
575         public Object getState() {
576             return state;
577         }
578
579         public List<ReplicatedLogEntry> getUnAppliedEntries() {
580             return unAppliedEntries;
581         }
582
583         public long getLastTerm() {
584             return lastTerm;
585         }
586
587         public long getLastAppliedIndex() {
588             return lastAppliedIndex;
589         }
590
591         public long getLastAppliedTerm() {
592             return lastAppliedTerm;
593         }
594     }
595
596     private class ElectionTermImpl implements ElectionTerm {
597         /**
598          * Identifier of the actor whose election term information this is
599          */
600         private long currentTerm = 0;
601         private String votedFor = null;
602
603         public long getCurrentTerm() {
604             return currentTerm;
605         }
606
607         public String getVotedFor() {
608             return votedFor;
609         }
610
611         @Override public void update(long currentTerm, String votedFor) {
612             LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
613
614             this.currentTerm = currentTerm;
615             this.votedFor = votedFor;
616         }
617
618         @Override
619         public void updateAndPersist(long currentTerm, String votedFor){
620             update(currentTerm, votedFor);
621             // FIXME : Maybe first persist then update the state
622             persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
623
624                 @Override public void apply(UpdateElectionTerm param)
625                     throws Exception {
626
627                 }
628             });
629         }
630     }
631
632     private static class UpdateElectionTerm implements Serializable {
633         private final long currentTerm;
634         private final String votedFor;
635
636         public UpdateElectionTerm(long currentTerm, String votedFor) {
637             this.currentTerm = currentTerm;
638             this.votedFor = votedFor;
639         }
640
641         public long getCurrentTerm() {
642             return currentTerm;
643         }
644
645         public String getVotedFor() {
646             return votedFor;
647         }
648     }
649
650 }