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