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