Raft behavior logging
[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.japi.Procedure;
14 import akka.persistence.RecoveryCompleted;
15 import akka.persistence.SaveSnapshotFailure;
16 import akka.persistence.SaveSnapshotSuccess;
17 import akka.persistence.SnapshotOffer;
18 import akka.persistence.SnapshotSelectionCriteria;
19 import com.google.common.annotations.VisibleForTesting;
20 import com.google.common.base.Optional;
21 import com.google.common.base.Stopwatch;
22 import com.google.protobuf.ByteString;
23 import java.io.Serializable;
24 import java.util.Map;
25 import org.opendaylight.controller.cluster.DataPersistenceProvider;
26 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
27 import org.opendaylight.controller.cluster.notifications.RoleChanged;
28 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
29 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
30 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
31 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
32 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
33 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
34 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
35 import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
36 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
37 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
38 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
39 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
40 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
41 import org.slf4j.Logger;
42 import org.slf4j.LoggerFactory;
43
44 /**
45  * RaftActor encapsulates a state machine that needs to be kept synchronized
46  * in a cluster. It implements the RAFT algorithm as described in the paper
47  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
48  * In Search of an Understandable Consensus Algorithm</a>
49  * <p/>
50  * RaftActor has 3 states and each state has a certain behavior associated
51  * with it. A Raft actor can behave as,
52  * <ul>
53  * <li> A Leader </li>
54  * <li> A Follower (or) </li>
55  * <li> A Candidate </li>
56  * </ul>
57  * <p/>
58  * <p/>
59  * A RaftActor MUST be a Leader in order to accept requests from clients to
60  * change the state of it's encapsulated state machine. Once a RaftActor becomes
61  * a Leader it is also responsible for ensuring that all followers ultimately
62  * have the same log and therefore the same state machine as itself.
63  * <p/>
64  * <p/>
65  * The current behavior of a RaftActor determines how election for leadership
66  * is initiated and how peer RaftActors react to request for votes.
67  * <p/>
68  * <p/>
69  * Each RaftActor also needs to know the current election term. It uses this
70  * information for a couple of things. One is to simply figure out who it
71  * voted for in the last election. Another is to figure out if the message
72  * it received to update it's state is stale.
73  * <p/>
74  * <p/>
75  * The RaftActor uses akka-persistence to store it's replicated log.
76  * Furthermore through it's behaviors a Raft Actor determines
77  * <p/>
78  * <ul>
79  * <li> when a log entry should be persisted </li>
80  * <li> when a log entry should be applied to the state machine (and) </li>
81  * <li> when a snapshot should be saved </li>
82  * </ul>
83  */
84 public abstract class RaftActor extends AbstractUntypedPersistentActor {
85     protected final Logger LOG = LoggerFactory.getLogger(getClass());
86
87     /**
88      * The current state determines the current behavior of a RaftActor
89      * A Raft Actor always starts off in the Follower State
90      */
91     private RaftActorBehavior currentBehavior;
92
93     /**
94      * This context should NOT be passed directly to any other actor it is
95      * only to be consumed by the RaftActorBehaviors
96      */
97     private final RaftActorContext context;
98
99     /**
100      * The in-memory journal
101      */
102     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
103
104     private CaptureSnapshot captureSnapshot = null;
105
106     private Stopwatch recoveryTimer;
107
108     private int currentRecoveryBatchCount;
109
110     public RaftActor(String id, Map<String, String> peerAddresses) {
111         this(id, peerAddresses, Optional.<ConfigParams>absent());
112     }
113
114     public RaftActor(String id, Map<String, String> peerAddresses,
115          Optional<ConfigParams> configParams) {
116
117         context = new RaftActorContextImpl(this.getSelf(),
118             this.getContext(), id, new ElectionTermImpl(),
119             -1, -1, replicatedLog, peerAddresses,
120             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
121             LOG);
122     }
123
124     private void initRecoveryTimer() {
125         if(recoveryTimer == null) {
126             recoveryTimer = Stopwatch.createStarted();
127         }
128     }
129
130     @Override
131     public void preStart() throws Exception {
132         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
133                 context.getConfigParams().getJournalRecoveryLogBatchSize());
134
135         super.preStart();
136     }
137
138     @Override
139     public void handleRecover(Object message) {
140         if(persistence().isRecoveryApplicable()) {
141             if (message instanceof SnapshotOffer) {
142                 onRecoveredSnapshot((SnapshotOffer) message);
143             } else if (message instanceof ReplicatedLogEntry) {
144                 onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
145             } else if (message instanceof ApplyLogEntries) {
146                 onRecoveredApplyLogEntries((ApplyLogEntries) message);
147             } else if (message instanceof DeleteEntries) {
148                 replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
149             } else if (message instanceof UpdateElectionTerm) {
150                 context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
151                         ((UpdateElectionTerm) message).getVotedFor());
152             } else if (message instanceof RecoveryCompleted) {
153                 onRecoveryCompletedMessage();
154             }
155         } else {
156             if (message instanceof RecoveryCompleted) {
157                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
158                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
159                 deleteMessages(lastSequenceNr());
160
161                 // Delete all the akka snapshots as they will not be needed
162                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
163
164                 onRecoveryComplete();
165
166                 initializeBehavior();
167             }
168         }
169     }
170
171     private void onRecoveredSnapshot(SnapshotOffer offer) {
172         if(LOG.isDebugEnabled()) {
173             LOG.debug("{}: SnapshotOffer called..", persistenceId());
174         }
175
176         initRecoveryTimer();
177
178         Snapshot snapshot = (Snapshot) offer.snapshot();
179
180         // Create a replicated log with the snapshot information
181         // The replicated log can be used later on to retrieve this snapshot
182         // when we need to install it on a peer
183         replicatedLog = new ReplicatedLogImpl(snapshot);
184
185         context.setReplicatedLog(replicatedLog);
186         context.setLastApplied(snapshot.getLastAppliedIndex());
187         context.setCommitIndex(snapshot.getLastAppliedIndex());
188
189         Stopwatch timer = Stopwatch.createStarted();
190
191         // Apply the snapshot to the actors state
192         applyRecoverySnapshot(snapshot.getState());
193
194         timer.stop();
195         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
196                 replicatedLog.size(), persistenceId(), timer.toString(),
197                 replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm());
198     }
199
200     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
201         if(LOG.isDebugEnabled()) {
202             LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
203         }
204
205         replicatedLog.append(logEntry);
206     }
207
208     private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
209         if(LOG.isDebugEnabled()) {
210             LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
211                     persistenceId(), context.getLastApplied() + 1, ale.getToIndex());
212         }
213
214         for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
215             batchRecoveredLogEntry(replicatedLog.get(i));
216         }
217
218         context.setLastApplied(ale.getToIndex());
219         context.setCommitIndex(ale.getToIndex());
220     }
221
222     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
223         initRecoveryTimer();
224
225         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
226         if(currentRecoveryBatchCount == 0) {
227             startLogRecoveryBatch(batchSize);
228         }
229
230         appendRecoveredLogEntry(logEntry.getData());
231
232         if(++currentRecoveryBatchCount >= batchSize) {
233             endCurrentLogRecoveryBatch();
234         }
235     }
236
237     private void endCurrentLogRecoveryBatch() {
238         applyCurrentLogRecoveryBatch();
239         currentRecoveryBatchCount = 0;
240     }
241
242     private void onRecoveryCompletedMessage() {
243         if(currentRecoveryBatchCount > 0) {
244             endCurrentLogRecoveryBatch();
245         }
246
247         onRecoveryComplete();
248
249         String recoveryTime = "";
250         if(recoveryTimer != null) {
251             recoveryTimer.stop();
252             recoveryTime = " in " + recoveryTimer.toString();
253             recoveryTimer = null;
254         }
255
256         LOG.info(
257             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
258                 "Persistence Id =  " + persistenceId() +
259                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
260                 "journal-size={}",
261             replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(),
262             replicatedLog.getSnapshotTerm(), replicatedLog.size());
263
264         initializeBehavior();
265     }
266
267     protected void initializeBehavior(){
268         changeCurrentBehavior(new Follower(context));
269     }
270
271     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
272         RaftActorBehavior oldBehavior = currentBehavior;
273         currentBehavior = newBehavior;
274         handleBehaviorChange(oldBehavior, currentBehavior);
275     }
276
277     @Override public void handleCommand(Object message) {
278         if (message instanceof ApplyState){
279             ApplyState applyState = (ApplyState) message;
280
281             if(LOG.isDebugEnabled()) {
282                 LOG.debug("{}: Applying state for log index {} data {}",
283                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
284                     applyState.getReplicatedLogEntry().getData());
285             }
286
287             applyState(applyState.getClientActor(), applyState.getIdentifier(),
288                 applyState.getReplicatedLogEntry().getData());
289
290         } else if (message instanceof ApplyLogEntries){
291             ApplyLogEntries ale = (ApplyLogEntries) message;
292             if(LOG.isDebugEnabled()) {
293                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), ale.getToIndex());
294             }
295             persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
296                 @Override
297                 public void apply(ApplyLogEntries param) throws Exception {
298                 }
299             });
300
301         } else if(message instanceof ApplySnapshot ) {
302             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
303
304             if(LOG.isDebugEnabled()) {
305                 LOG.debug("{}: ApplySnapshot called on Follower Actor " +
306                         "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
307                     snapshot.getLastAppliedTerm()
308                 );
309             }
310
311             applySnapshot(snapshot.getState());
312
313             //clears the followers log, sets the snapshot index to ensure adjusted-index works
314             replicatedLog = new ReplicatedLogImpl(snapshot);
315             context.setReplicatedLog(replicatedLog);
316             context.setLastApplied(snapshot.getLastAppliedIndex());
317
318         } else if (message instanceof FindLeader) {
319             getSender().tell(
320                 new FindLeaderReply(getLeaderAddress()),
321                 getSelf()
322             );
323
324         } else if (message instanceof SaveSnapshotSuccess) {
325             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
326             LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
327
328             long sequenceNumber = success.metadata().sequenceNr();
329
330             commitSnapshot(sequenceNumber);
331
332         } else if (message instanceof SaveSnapshotFailure) {
333             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
334
335             LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
336                     persistenceId(), saveSnapshotFailure.cause());
337
338             context.getReplicatedLog().snapshotRollback();
339
340             LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
341                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
342                 context.getReplicatedLog().getSnapshotIndex(),
343                 context.getReplicatedLog().getSnapshotTerm(),
344                 context.getReplicatedLog().size());
345
346         } else if (message instanceof CaptureSnapshot) {
347             LOG.info("{}: CaptureSnapshot received by actor", persistenceId());
348
349             if(captureSnapshot == null) {
350                 captureSnapshot = (CaptureSnapshot)message;
351                 createSnapshot();
352             }
353
354         } else if (message instanceof CaptureSnapshotReply){
355             handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
356
357         } else {
358             RaftActorBehavior oldBehavior = currentBehavior;
359             currentBehavior = currentBehavior.handleMessage(getSender(), message);
360
361             handleBehaviorChange(oldBehavior, currentBehavior);
362         }
363     }
364
365     private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) {
366         if (oldBehavior != currentBehavior){
367             onStateChanged();
368         }
369
370         String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId();
371         String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name();
372
373         // it can happen that the state has not changed but the leader has changed.
374         onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
375
376         if (getRoleChangeNotifier().isPresent() &&
377                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
378             getRoleChangeNotifier().get().tell(
379                     new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()),
380                     getSelf());
381         }
382     }
383
384     /**
385      * When a derived RaftActor needs to persist something it must call
386      * persistData.
387      *
388      * @param clientActor
389      * @param identifier
390      * @param data
391      */
392     protected void persistData(final ActorRef clientActor, final String identifier,
393         final Payload data) {
394
395         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
396             context.getReplicatedLog().lastIndex() + 1,
397             context.getTermInformation().getCurrentTerm(), data);
398
399         if(LOG.isDebugEnabled()) {
400             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
401         }
402
403         final RaftActorContext raftContext = getRaftActorContext();
404
405         replicatedLog
406                 .appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
407                     @Override
408                     public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
409                         if(!hasFollowers()){
410                             // Increment the Commit Index and the Last Applied values
411                             raftContext.setCommitIndex(replicatedLogEntry.getIndex());
412                             raftContext.setLastApplied(replicatedLogEntry.getIndex());
413
414                             // Apply the state immediately
415                             applyState(clientActor, identifier, data);
416
417                             // Send a ApplyLogEntries message so that we write the fact that we applied
418                             // the state to durable storage
419                             self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self());
420
421                             // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot
422                             if(!context.isSnapshotCaptureInitiated()){
423                                 raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(),
424                                         raftContext.getTermInformation().getCurrentTerm());
425                                 raftContext.getReplicatedLog().snapshotCommit();
426                             } else {
427                                 LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress",
428                                         persistenceId(), getId());
429                             }
430                         } else if (clientActor != null) {
431                             // Send message for replication
432                             currentBehavior.handleMessage(getSelf(),
433                                     new Replicate(clientActor, identifier,
434                                             replicatedLogEntry)
435                             );
436                         }
437
438                     }
439                 });    }
440
441     protected String getId() {
442         return context.getId();
443     }
444
445     /**
446      * Derived actors can call the isLeader method to check if the current
447      * RaftActor is the Leader or not
448      *
449      * @return true it this RaftActor is a Leader false otherwise
450      */
451     protected boolean isLeader() {
452         return context.getId().equals(currentBehavior.getLeaderId());
453     }
454
455     /**
456      * Derived actor can call getLeader if they need a reference to the Leader.
457      * This would be useful for example in forwarding a request to an actor
458      * which is the leader
459      *
460      * @return A reference to the leader if known, null otherwise
461      */
462     protected ActorSelection getLeader(){
463         String leaderAddress = getLeaderAddress();
464
465         if(leaderAddress == null){
466             return null;
467         }
468
469         return context.actorSelection(leaderAddress);
470     }
471
472     /**
473      *
474      * @return the current leader's id
475      */
476     protected String getLeaderId(){
477         return currentBehavior.getLeaderId();
478     }
479
480     protected RaftState getRaftState() {
481         return currentBehavior.state();
482     }
483
484     protected ReplicatedLogEntry getLastLogEntry() {
485         return replicatedLog.last();
486     }
487
488     protected Long getCurrentTerm(){
489         return context.getTermInformation().getCurrentTerm();
490     }
491
492     protected Long getCommitIndex(){
493         return context.getCommitIndex();
494     }
495
496     protected Long getLastApplied(){
497         return context.getLastApplied();
498     }
499
500     protected RaftActorContext getRaftActorContext() {
501         return context;
502     }
503
504     /**
505      * setPeerAddress sets the address of a known peer at a later time.
506      * <p>
507      * This is to account for situations where a we know that a peer
508      * exists but we do not know an address up-front. This may also be used in
509      * situations where a known peer starts off in a different location and we
510      * need to change it's address
511      * <p>
512      * Note that if the peerId does not match the list of peers passed to
513      * this actor during construction an IllegalStateException will be thrown.
514      *
515      * @param peerId
516      * @param peerAddress
517      */
518     protected void setPeerAddress(String peerId, String peerAddress){
519         context.setPeerAddress(peerId, peerAddress);
520     }
521
522     protected void commitSnapshot(long sequenceNumber) {
523         context.getReplicatedLog().snapshotCommit();
524
525         // TODO: Not sure if we want to be this aggressive with trimming stuff
526         trimPersistentData(sequenceNumber);
527     }
528
529     /**
530      * The applyState method will be called by the RaftActor when some data
531      * needs to be applied to the actor's state
532      *
533      * @param clientActor A reference to the client who sent this message. This
534      *                    is the same reference that was passed to persistData
535      *                    by the derived actor. clientActor may be null when
536      *                    the RaftActor is behaving as a follower or during
537      *                    recovery.
538      * @param identifier  The identifier of the persisted data. This is also
539      *                    the same identifier that was passed to persistData by
540      *                    the derived actor. identifier may be null when
541      *                    the RaftActor is behaving as a follower or during
542      *                    recovery
543      * @param data        A piece of data that was persisted by the persistData call.
544      *                    This should NEVER be null.
545      */
546     protected abstract void applyState(ActorRef clientActor, String identifier,
547         Object data);
548
549     /**
550      * This method is called during recovery at the start of a batch of state entries. Derived
551      * classes should perform any initialization needed to start a batch.
552      */
553     protected abstract void startLogRecoveryBatch(int maxBatchSize);
554
555     /**
556      * This method is called during recovery to append state data to the current batch. This method
557      * is called 1 or more times after {@link #startLogRecoveryBatch}.
558      *
559      * @param data the state data
560      */
561     protected abstract void appendRecoveredLogEntry(Payload data);
562
563     /**
564      * This method is called during recovery to reconstruct the state of the actor.
565      *
566      * @param snapshotBytes A snapshot of the state of the actor
567      */
568     protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
569
570     /**
571      * This method is called during recovery at the end of a batch to apply the current batched
572      * log entries. This method is called after {@link #appendRecoveredLogEntry}.
573      */
574     protected abstract void applyCurrentLogRecoveryBatch();
575
576     /**
577      * This method is called when recovery is complete.
578      */
579     protected abstract void onRecoveryComplete();
580
581     /**
582      * This method will be called by the RaftActor when a snapshot needs to be
583      * created. The derived actor should respond with its current state.
584      * <p/>
585      * During recovery the state that is returned by the derived actor will
586      * be passed back to it by calling the applySnapshot  method
587      *
588      * @return The current state of the actor
589      */
590     protected abstract void createSnapshot();
591
592     /**
593      * This method can be called at any other point during normal
594      * operations when the derived actor is out of sync with it's peers
595      * and the only way to bring it in sync is by applying a snapshot
596      *
597      * @param snapshotBytes A snapshot of the state of the actor
598      */
599     protected abstract void applySnapshot(byte[] snapshotBytes);
600
601     /**
602      * This method will be called by the RaftActor when the state of the
603      * RaftActor changes. The derived actor can then use methods like
604      * isLeader or getLeader to do something useful
605      */
606     protected abstract void onStateChanged();
607
608     protected abstract DataPersistenceProvider persistence();
609
610     /**
611      * Notifier Actor for this RaftActor to notify when a role change happens
612      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
613      */
614     protected abstract Optional<ActorRef> getRoleChangeNotifier();
615
616     protected void onLeaderChanged(String oldLeader, String newLeader){};
617
618     private void trimPersistentData(long sequenceNumber) {
619         // Trim akka snapshots
620         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
621         // For now guessing that it is ANDed.
622         persistence().deleteSnapshots(new SnapshotSelectionCriteria(
623             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
624
625         // Trim akka journal
626         persistence().deleteMessages(sequenceNumber);
627     }
628
629     private String getLeaderAddress(){
630         if(isLeader()){
631             return getSelf().path().toString();
632         }
633         String leaderId = currentBehavior.getLeaderId();
634         if (leaderId == null) {
635             return null;
636         }
637         String peerAddress = context.getPeerAddress(leaderId);
638         if(LOG.isDebugEnabled()) {
639             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
640                     persistenceId(), leaderId, peerAddress);
641         }
642
643         return peerAddress;
644     }
645
646     private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
647         LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
648
649         // create a snapshot object from the state provided and save it
650         // when snapshot is saved async, SaveSnapshotSuccess is raised.
651
652         Snapshot sn = Snapshot.create(snapshotBytes,
653             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
654             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
655             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
656
657         persistence().saveSnapshot(sn);
658
659         LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
660
661         long dataThreshold = Runtime.getRuntime().totalMemory() *
662                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
663         if (context.getReplicatedLog().dataSize() > dataThreshold) {
664             // if memory is less, clear the log based on lastApplied.
665             // this could/should only happen if one of the followers is down
666             // as normally we keep removing from the log when its replicated to all.
667             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
668                     captureSnapshot.getLastAppliedTerm());
669
670         } else {
671             // clear the log based on replicatedToAllIndex
672             context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
673                     captureSnapshot.getReplicatedToAllTerm());
674         }
675         getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
676
677         LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
678             "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
679             captureSnapshot.getLastAppliedTerm());
680
681         if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) {
682             // this would be call straight to the leader and won't initiate in serialization
683             currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(
684                     ByteString.copyFrom(snapshotBytes)));
685         }
686
687         captureSnapshot = null;
688         context.setSnapshotCaptureInitiated(false);
689     }
690
691     protected boolean hasFollowers(){
692         return getRaftActorContext().getPeerAddresses().keySet().size() > 0;
693     }
694
695     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
696
697         private static final int DATA_SIZE_DIVIDER = 5;
698         private long dataSizeSinceLastSnapshot = 0;
699
700         public ReplicatedLogImpl(Snapshot snapshot) {
701             super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
702                 snapshot.getUnAppliedEntries());
703         }
704
705         public ReplicatedLogImpl() {
706             super();
707         }
708
709         @Override public void removeFromAndPersist(long logEntryIndex) {
710             int adjustedIndex = adjustedIndex(logEntryIndex);
711
712             if (adjustedIndex < 0) {
713                 return;
714             }
715
716             // FIXME: Maybe this should be done after the command is saved
717             journal.subList(adjustedIndex , journal.size()).clear();
718
719             persistence().persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>() {
720
721                 @Override
722                 public void apply(DeleteEntries param)
723                         throws Exception {
724                     //FIXME : Doing nothing for now
725                     dataSize = 0;
726                     for (ReplicatedLogEntry entry : journal) {
727                         dataSize += entry.size();
728                     }
729                 }
730             });
731         }
732
733         @Override public void appendAndPersist(
734             final ReplicatedLogEntry replicatedLogEntry) {
735             appendAndPersist(replicatedLogEntry, null);
736         }
737
738         public void appendAndPersist(
739             final ReplicatedLogEntry replicatedLogEntry,
740             final Procedure<ReplicatedLogEntry> callback)  {
741
742             if(LOG.isDebugEnabled()) {
743                 LOG.debug("{}: Append log entry and persist {} ", persistenceId(), replicatedLogEntry);
744             }
745
746             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
747             journal.add(replicatedLogEntry);
748
749             // When persisting events with persist it is guaranteed that the
750             // persistent actor will not receive further commands between the
751             // persist call and the execution(s) of the associated event
752             // handler. This also holds for multiple persist calls in context
753             // of a single command.
754             persistence().persist(replicatedLogEntry,
755                 new Procedure<ReplicatedLogEntry>() {
756                     @Override
757                     public void apply(ReplicatedLogEntry evt) throws Exception {
758                         int logEntrySize = replicatedLogEntry.size();
759
760                         dataSize += logEntrySize;
761                         long dataSizeForCheck = dataSize;
762
763                         dataSizeSinceLastSnapshot += logEntrySize;
764                         long journalSize = lastIndex() + 1;
765
766                         if(!hasFollowers()) {
767                             // When we do not have followers we do not maintain an in-memory log
768                             // due to this the journalSize will never become anything close to the
769                             // snapshot batch count. In fact will mostly be 1.
770                             // Similarly since the journal's dataSize depends on the entries in the
771                             // journal the journal's dataSize will never reach a value close to the
772                             // memory threshold.
773                             // By maintaining the dataSize outside the journal we are tracking essentially
774                             // what we have written to the disk however since we no longer are in
775                             // need of doing a snapshot just for the sake of freeing up memory we adjust
776                             // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often
777                             // as if we were maintaining a real snapshot
778                             dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER;
779                         }
780
781                         long dataThreshold = Runtime.getRuntime().totalMemory() *
782                                 getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100;
783
784                         // when a snaphsot is being taken, captureSnapshot != null
785                         if (!context.isSnapshotCaptureInitiated() &&
786                                 ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 ||
787                                         dataSizeForCheck > dataThreshold)) {
788
789                             dataSizeSinceLastSnapshot = 0;
790
791                             LOG.info("{}: Initiating Snapshot Capture..", persistenceId());
792                             long lastAppliedIndex = -1;
793                             long lastAppliedTerm = -1;
794
795                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
796                             if (!hasFollowers()) {
797                                 lastAppliedIndex = replicatedLogEntry.getIndex();
798                                 lastAppliedTerm = replicatedLogEntry.getTerm();
799                             } else if (lastAppliedEntry != null) {
800                                 lastAppliedIndex = lastAppliedEntry.getIndex();
801                                 lastAppliedTerm = lastAppliedEntry.getTerm();
802                             }
803
804                             if(LOG.isDebugEnabled()) {
805                                 LOG.debug("{}: Snapshot Capture logSize: {}", persistenceId(), journal.size());
806                                 LOG.debug("{}: Snapshot Capture lastApplied:{} ",
807                                         persistenceId(), context.getLastApplied());
808                                 LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(),
809                                         lastAppliedIndex);
810                                 LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(),
811                                         lastAppliedTerm);
812                             }
813
814                             // send a CaptureSnapshot to self to make the expensive operation async.
815                             long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex();
816                             ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex);
817                             getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm,
818                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1),
819                                 (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)),
820                                 null);
821                             context.setSnapshotCaptureInitiated(true);
822                         }
823                         if (callback != null){
824                             callback.apply(replicatedLogEntry);
825                         }
826                     }
827                 }
828             );
829         }
830
831     }
832
833     static class DeleteEntries implements Serializable {
834         private static final long serialVersionUID = 1L;
835         private final int fromIndex;
836
837         public DeleteEntries(int fromIndex) {
838             this.fromIndex = fromIndex;
839         }
840
841         public int getFromIndex() {
842             return fromIndex;
843         }
844     }
845
846
847     private class ElectionTermImpl implements ElectionTerm {
848         /**
849          * Identifier of the actor whose election term information this is
850          */
851         private long currentTerm = 0;
852         private String votedFor = null;
853
854         @Override
855         public long getCurrentTerm() {
856             return currentTerm;
857         }
858
859         @Override
860         public String getVotedFor() {
861             return votedFor;
862         }
863
864         @Override public void update(long currentTerm, String votedFor) {
865             if(LOG.isDebugEnabled()) {
866                 LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor);
867             }
868             this.currentTerm = currentTerm;
869             this.votedFor = votedFor;
870         }
871
872         @Override
873         public void updateAndPersist(long currentTerm, String votedFor){
874             update(currentTerm, votedFor);
875             // FIXME : Maybe first persist then update the state
876             persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
877
878                 @Override public void apply(UpdateElectionTerm param)
879                     throws Exception {
880
881                 }
882             });
883         }
884     }
885
886     static class UpdateElectionTerm implements Serializable {
887         private static final long serialVersionUID = 1L;
888         private final long currentTerm;
889         private final String votedFor;
890
891         public UpdateElectionTerm(long currentTerm, String votedFor) {
892             this.currentTerm = currentTerm;
893             this.votedFor = votedFor;
894         }
895
896         public long getCurrentTerm() {
897             return currentTerm;
898         }
899
900         public String getVotedFor() {
901             return votedFor;
902         }
903     }
904
905     protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider {
906
907         public NonPersistentRaftDataProvider(){
908
909         }
910
911         /**
912          * The way snapshotting works is,
913          * <ol>
914          * <li> RaftActor calls createSnapshot on the Shard
915          * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
916          * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save the snapshot.
917          * The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the RaftActor gets SaveSnapshot
918          * success it commits the snapshot to the in-memory journal. This commitSnapshot is mimicking what is done
919          * in SaveSnapshotSuccess.
920          * </ol>
921          * @param o
922          */
923         @Override
924         public void saveSnapshot(Object o) {
925             // Make saving Snapshot successful
926             commitSnapshot(-1L);
927         }
928     }
929
930     @VisibleForTesting
931     void setCurrentBehavior(AbstractRaftActorBehavior behavior) {
932         currentBehavior = behavior;
933     }
934
935     protected RaftActorBehavior getCurrentBehavior() {
936         return currentBehavior;
937     }
938
939 }