Merge "Fixed for bug 1197"
[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 org.opendaylight.controller.cluster.raft.behaviors.Candidate;
26 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
27 import org.opendaylight.controller.cluster.raft.behaviors.Leader;
28 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
29 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
30 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
31 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
32 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
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      * setPeerAddress sets the address of a known peer at a later time.
255      * <p>
256      * This is to account for situations where a we know that a peer
257      * exists but we do not know an address up-front. This may also be used in
258      * situations where a known peer starts off in a different location and we
259      * need to change it's address
260      * <p>
261      * Note that if the peerId does not match the list of peers passed to
262      * this actor during construction an IllegalStateException will be thrown.
263      *
264      * @param peerId
265      * @param peerAddress
266      */
267     protected void setPeerAddress(String peerId, String peerAddress){
268         context.setPeerAddress(peerId, peerAddress);
269     }
270
271
272
273     /**
274      * The applyState method will be called by the RaftActor when some data
275      * needs to be applied to the actor's state
276      *
277      * @param clientActor A reference to the client who sent this message. This
278      *                    is the same reference that was passed to persistData
279      *                    by the derived actor. clientActor may be null when
280      *                    the RaftActor is behaving as a follower or during
281      *                    recovery.
282      * @param identifier  The identifier of the persisted data. This is also
283      *                    the same identifier that was passed to persistData by
284      *                    the derived actor. identifier may be null when
285      *                    the RaftActor is behaving as a follower or during
286      *                    recovery
287      * @param data        A piece of data that was persisted by the persistData call.
288      *                    This should NEVER be null.
289      */
290     protected abstract void applyState(ActorRef clientActor, String identifier,
291         Object data);
292
293     /**
294      * This method will be called by the RaftActor when a snapshot needs to be
295      * created. The derived actor should respond with its current state.
296      * <p/>
297      * During recovery the state that is returned by the derived actor will
298      * be passed back to it by calling the applySnapshot  method
299      *
300      * @return The current state of the actor
301      */
302     protected abstract Object createSnapshot();
303
304     /**
305      * This method will be called by the RaftActor during recovery to
306      * reconstruct the state of the actor.
307      * <p/>
308      * This method may also be called at any other point during normal
309      * operations when the derived actor is out of sync with it's peers
310      * and the only way to bring it in sync is by applying a snapshot
311      *
312      * @param snapshot A snapshot of the state of the actor
313      */
314     protected abstract void applySnapshot(Object snapshot);
315
316     private RaftActorBehavior switchBehavior(RaftState state) {
317         if (currentBehavior != null) {
318             if (currentBehavior.state() == state) {
319                 return currentBehavior;
320             }
321             LOG.info("Switching from state " + currentBehavior.state() + " to "
322                 + state);
323
324             try {
325                 currentBehavior.close();
326             } catch (Exception e) {
327                 LOG.error(e,
328                     "Failed to close behavior : " + currentBehavior.state());
329             }
330
331         } else {
332             LOG.info("Switching behavior to " + state);
333         }
334         RaftActorBehavior behavior = null;
335         if (state == RaftState.Candidate) {
336             behavior = new Candidate(context);
337         } else if (state == RaftState.Follower) {
338             behavior = new Follower(context);
339         } else {
340             behavior = new Leader(context);
341         }
342         return behavior;
343     }
344
345     private void trimPersistentData(long sequenceNumber) {
346         // Trim snapshots
347         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
348         // For now guessing that it is ANDed.
349         deleteSnapshots(new SnapshotSelectionCriteria(
350             sequenceNumber - 100000, 43200000));
351
352         // Trim journal
353         deleteMessages(sequenceNumber);
354     }
355
356
357     private class ReplicatedLogImpl implements ReplicatedLog {
358         private final List<ReplicatedLogEntry> journal;
359         private final Object snapshot;
360         private long snapshotIndex = -1;
361         private long snapshotTerm = -1;
362
363         public ReplicatedLogImpl(Snapshot snapshot) {
364             this.snapshot = snapshot.getState();
365             this.snapshotIndex = snapshot.getLastAppliedIndex();
366             this.snapshotTerm = snapshot.getLastAppliedTerm();
367
368             this.journal = new ArrayList<>(snapshot.getUnAppliedEntries());
369         }
370
371         public ReplicatedLogImpl() {
372             this.snapshot = null;
373             this.journal = new ArrayList<>();
374         }
375
376         @Override public ReplicatedLogEntry get(long index) {
377             int adjustedIndex = adjustedIndex(index);
378
379             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
380                 return null;
381             }
382
383             return journal.get(adjustedIndex);
384         }
385
386         @Override public ReplicatedLogEntry last() {
387             if (journal.size() == 0) {
388                 return null;
389             }
390             return get(journal.size() - 1);
391         }
392
393         @Override public long lastIndex() {
394             if (journal.size() == 0) {
395                 return -1;
396             }
397
398             return last().getIndex();
399         }
400
401         @Override public long lastTerm() {
402             if (journal.size() == 0) {
403                 return -1;
404             }
405
406             return last().getTerm();
407         }
408
409
410         @Override public void removeFrom(long index) {
411             int adjustedIndex = adjustedIndex(index);
412
413             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
414                 return;
415             }
416
417             journal.subList(adjustedIndex , journal.size()).clear();
418         }
419
420
421         @Override public void removeFromAndPersist(long index) {
422             int adjustedIndex = adjustedIndex(index);
423
424             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
425                 return;
426             }
427
428             // FIXME: Maybe this should be done after the command is saved
429             journal.subList(adjustedIndex , journal.size()).clear();
430
431             persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
432
433                 @Override public void apply(DeleteEntries param)
434                     throws Exception {
435                     //FIXME : Doing nothing for now
436                 }
437             });
438
439
440         }
441
442         @Override public void append(
443             final ReplicatedLogEntry replicatedLogEntry) {
444             journal.add(replicatedLogEntry);
445         }
446
447         @Override public List<ReplicatedLogEntry> getFrom(long index) {
448             int adjustedIndex = adjustedIndex(index);
449
450             List<ReplicatedLogEntry> entries = new ArrayList<>(100);
451             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
452                 return entries;
453             }
454
455
456             for (int i = adjustedIndex;
457                  i < journal.size(); i++) {
458                 entries.add(journal.get(i));
459             }
460             return entries;
461         }
462
463         @Override public void appendAndPersist(
464             final ReplicatedLogEntry replicatedLogEntry) {
465             appendAndPersist(null, null, replicatedLogEntry);
466         }
467
468         public void appendAndPersist(final ActorRef clientActor,
469             final String identifier,
470             final ReplicatedLogEntry replicatedLogEntry) {
471             context.getLogger().debug(
472                 "Append log entry and persist {} ", replicatedLogEntry);
473             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
474             journal.add(replicatedLogEntry);
475
476             // When persisting events with persist it is guaranteed that the
477             // persistent actor will not receive further commands between the
478             // persist call and the execution(s) of the associated event
479             // handler. This also holds for multiple persist calls in context
480             // of a single command.
481             persist(replicatedLogEntry,
482                 new Procedure<ReplicatedLogEntry>() {
483                     public void apply(ReplicatedLogEntry evt) throws Exception {
484                         // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned.
485                         if (size() > 100000) {
486                             ReplicatedLogEntry lastAppliedEntry =
487                                 get(context.getLastApplied());
488                             long lastAppliedIndex = -1;
489                             long lastAppliedTerm = -1;
490                             if (lastAppliedEntry != null) {
491                                 lastAppliedIndex = lastAppliedEntry.getIndex();
492                                 lastAppliedTerm = lastAppliedEntry.getTerm();
493                             }
494
495                             saveSnapshot(Snapshot.create(createSnapshot(),
496                                 getFrom(context.getLastApplied() + 1),
497                                 lastIndex(), lastTerm(), lastAppliedIndex,
498                                 lastAppliedTerm));
499                         }
500                         // Send message for replication
501                         if (clientActor != null) {
502                             currentBehavior.handleMessage(getSelf(),
503                                 new Replicate(clientActor, identifier,
504                                     replicatedLogEntry)
505                             );
506                         }
507                     }
508                 }
509             );
510         }
511
512         @Override public long size() {
513             return journal.size() + snapshotIndex + 1;
514         }
515
516         @Override public boolean isPresent(long index) {
517             int adjustedIndex = adjustedIndex(index);
518
519             if (adjustedIndex < 0 || adjustedIndex >= journal.size()) {
520                 return false;
521             }
522             return true;
523         }
524
525         @Override public boolean isInSnapshot(long index) {
526             return index <= snapshotIndex;
527         }
528
529         @Override public Object getSnapshot() {
530             return snapshot;
531         }
532
533         @Override public long getSnapshotIndex() {
534             return snapshotIndex;
535         }
536
537         @Override public long getSnapshotTerm() {
538             return snapshotTerm;
539         }
540
541         private int adjustedIndex(long index) {
542             if(snapshotIndex < 0){
543                 return (int) index;
544             }
545             return (int) (index - snapshotIndex);
546         }
547     }
548
549
550
551
552     private static class DeleteEntries implements Serializable {
553         private final int fromIndex;
554
555
556         public DeleteEntries(int fromIndex) {
557             this.fromIndex = fromIndex;
558         }
559
560         public int getFromIndex() {
561             return fromIndex;
562         }
563     }
564
565
566     private static class Snapshot implements Serializable {
567         private final Object state;
568         private final List<ReplicatedLogEntry> unAppliedEntries;
569         private final long lastIndex;
570         private final long lastTerm;
571         private final long lastAppliedIndex;
572         private final long lastAppliedTerm;
573
574         private Snapshot(Object state,
575             List<ReplicatedLogEntry> unAppliedEntries, long lastIndex,
576             long lastTerm, long lastAppliedIndex, long lastAppliedTerm) {
577             this.state = state;
578             this.unAppliedEntries = unAppliedEntries;
579             this.lastIndex = lastIndex;
580             this.lastTerm = lastTerm;
581             this.lastAppliedIndex = lastAppliedIndex;
582             this.lastAppliedTerm = lastAppliedTerm;
583         }
584
585
586         public static Snapshot create(Object state,
587             List<ReplicatedLogEntry> entries, long lastIndex, long lastTerm,
588             long lastAppliedIndex, long lastAppliedTerm) {
589             return new Snapshot(state, entries, lastIndex, lastTerm,
590                 lastAppliedIndex, lastAppliedTerm);
591         }
592
593         public Object getState() {
594             return state;
595         }
596
597         public List<ReplicatedLogEntry> getUnAppliedEntries() {
598             return unAppliedEntries;
599         }
600
601         public long getLastTerm() {
602             return lastTerm;
603         }
604
605         public long getLastAppliedIndex() {
606             return lastAppliedIndex;
607         }
608
609         public long getLastAppliedTerm() {
610             return lastAppliedTerm;
611         }
612     }
613
614     private class ElectionTermImpl implements ElectionTerm {
615         /**
616          * Identifier of the actor whose election term information this is
617          */
618         private long currentTerm = 0;
619         private String votedFor = null;
620
621         public long getCurrentTerm() {
622             return currentTerm;
623         }
624
625         public String getVotedFor() {
626             return votedFor;
627         }
628
629         @Override public void update(long currentTerm, String votedFor) {
630             LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
631
632             this.currentTerm = currentTerm;
633             this.votedFor = votedFor;
634         }
635
636         @Override
637         public void updateAndPersist(long currentTerm, String votedFor){
638             update(currentTerm, votedFor);
639             // FIXME : Maybe first persist then update the state
640             persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
641
642                 @Override public void apply(UpdateElectionTerm param)
643                     throws Exception {
644
645                 }
646             });
647         }
648     }
649
650     private static class UpdateElectionTerm implements Serializable {
651         private final long currentTerm;
652         private final String votedFor;
653
654         public UpdateElectionTerm(long currentTerm, String votedFor) {
655             this.currentTerm = currentTerm;
656             this.votedFor = votedFor;
657         }
658
659         public long getCurrentTerm() {
660             return currentTerm;
661         }
662
663         public String getVotedFor() {
664             return votedFor;
665         }
666     }
667
668 }