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