Merge "Cleanup RpcRoutingStrategy definition"
[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 com.google.common.base.Optional;
23 import com.google.common.base.Stopwatch;
24 import com.google.protobuf.ByteString;
25 import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
26 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
27 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
28 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
29 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
30 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
31 import org.opendaylight.controller.cluster.raft.base.messages.SendHeartBeat;
32 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
33 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
34 import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer;
35 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
36 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
37 import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer;
38 import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply;
39 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
40 import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages;
41 import java.io.Serializable;
42 import java.util.Map;
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 UntypedPersistentActor {
85     protected final LoggingAdapter LOG =
86         Logging.getLogger(getContext().system(), this);
87
88     /**
89      * The current state determines the current behavior of a RaftActor
90      * A Raft Actor always starts off in the Follower State
91      */
92     private RaftActorBehavior currentBehavior;
93
94     /**
95      * This context should NOT be passed directly to any other actor it is
96      * only to be consumed by the RaftActorBehaviors
97      */
98     private final RaftActorContext context;
99
100     /**
101      * The in-memory journal
102      */
103     private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl();
104
105     private CaptureSnapshot captureSnapshot = null;
106
107     private volatile boolean hasSnapshotCaptureInitiated = false;
108
109     private Stopwatch recoveryTimer;
110
111     private int currentRecoveryBatchCount;
112
113     public RaftActor(String id, Map<String, String> peerAddresses) {
114         this(id, peerAddresses, Optional.<ConfigParams>absent());
115     }
116
117     public RaftActor(String id, Map<String, String> peerAddresses,
118          Optional<ConfigParams> configParams) {
119
120         context = new RaftActorContextImpl(this.getSelf(),
121             this.getContext(), id, new ElectionTermImpl(),
122             -1, -1, replicatedLog, peerAddresses,
123             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
124             LOG);
125     }
126
127     private void initRecoveryTimer() {
128         if(recoveryTimer == null) {
129             recoveryTimer = new Stopwatch();
130             recoveryTimer.start();
131         }
132     }
133
134     @Override
135     public void preStart() throws Exception {
136         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
137                 context.getConfigParams().getJournalRecoveryLogBatchSize());
138         super.preStart();
139     }
140
141     @Override
142     public void onReceiveRecover(Object message) {
143         if (message instanceof SnapshotOffer) {
144             onRecoveredSnapshot((SnapshotOffer)message);
145         } else if (message instanceof ReplicatedLogEntry) {
146             onRecoveredJournalLogEntry((ReplicatedLogEntry)message);
147         } else if (message instanceof ApplyLogEntries) {
148             onRecoveredApplyLogEntries((ApplyLogEntries)message);
149         } else if (message instanceof DeleteEntries) {
150             replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex());
151         } else if (message instanceof UpdateElectionTerm) {
152             context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
153                     ((UpdateElectionTerm) message).getVotedFor());
154         } else if (message instanceof RecoveryCompleted) {
155             onRecoveryCompletedMessage();
156         }
157     }
158
159     private void onRecoveredSnapshot(SnapshotOffer offer) {
160         if(LOG.isDebugEnabled()) {
161             LOG.debug("SnapshotOffer called..");
162         }
163
164         initRecoveryTimer();
165
166         Snapshot snapshot = (Snapshot) offer.snapshot();
167
168         // Create a replicated log with the snapshot information
169         // The replicated log can be used later on to retrieve this snapshot
170         // when we need to install it on a peer
171         replicatedLog = new ReplicatedLogImpl(snapshot);
172
173         context.setReplicatedLog(replicatedLog);
174         context.setLastApplied(snapshot.getLastAppliedIndex());
175         context.setCommitIndex(snapshot.getLastAppliedIndex());
176
177         Stopwatch timer = new Stopwatch();
178         timer.start();
179
180         // Apply the snapshot to the actors state
181         applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState()));
182
183         timer.stop();
184         LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
185                 replicatedLog.size(), persistenceId(), timer.toString(),
186                 replicatedLog.snapshotIndex, replicatedLog.snapshotTerm);
187     }
188
189     private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
190         if(LOG.isDebugEnabled()) {
191             LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex());
192         }
193
194         replicatedLog.append(logEntry);
195     }
196
197     private void onRecoveredApplyLogEntries(ApplyLogEntries ale) {
198         if(LOG.isDebugEnabled()) {
199             LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}",
200                     context.getLastApplied() + 1, ale.getToIndex());
201         }
202
203         for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) {
204             batchRecoveredLogEntry(replicatedLog.get(i));
205         }
206
207         context.setLastApplied(ale.getToIndex());
208         context.setCommitIndex(ale.getToIndex());
209     }
210
211     private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
212         initRecoveryTimer();
213
214         int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
215         if(currentRecoveryBatchCount == 0) {
216             startLogRecoveryBatch(batchSize);
217         }
218
219         appendRecoveredLogEntry(logEntry.getData());
220
221         if(++currentRecoveryBatchCount >= batchSize) {
222             endCurrentLogRecoveryBatch();
223         }
224     }
225
226     private void endCurrentLogRecoveryBatch() {
227         applyCurrentLogRecoveryBatch();
228         currentRecoveryBatchCount = 0;
229     }
230
231     private void onRecoveryCompletedMessage() {
232         if(currentRecoveryBatchCount > 0) {
233             endCurrentLogRecoveryBatch();
234         }
235
236         onRecoveryComplete();
237
238         String recoveryTime = "";
239         if(recoveryTimer != null) {
240             recoveryTimer.stop();
241             recoveryTime = " in " + recoveryTimer.toString();
242             recoveryTimer = null;
243         }
244
245         LOG.info(
246             "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
247                 "Persistence Id =  " + persistenceId() +
248                 " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
249                 "journal-size={}",
250             replicatedLog.lastIndex(), replicatedLog.snapshotIndex,
251             replicatedLog.snapshotTerm, replicatedLog.size());
252
253         currentBehavior = new Follower(context);
254         onStateChanged();
255     }
256
257     @Override public void onReceiveCommand(Object message) {
258         if (message instanceof ApplyState){
259             ApplyState applyState = (ApplyState) message;
260
261             if(LOG.isDebugEnabled()) {
262                 LOG.debug("Applying state for log index {} data {}",
263                     applyState.getReplicatedLogEntry().getIndex(),
264                     applyState.getReplicatedLogEntry().getData());
265             }
266
267             applyState(applyState.getClientActor(), applyState.getIdentifier(),
268                 applyState.getReplicatedLogEntry().getData());
269
270         } else if (message instanceof ApplyLogEntries){
271             ApplyLogEntries ale = (ApplyLogEntries) message;
272             if(LOG.isDebugEnabled()) {
273                 LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex());
274             }
275             persist(new ApplyLogEntries(ale.getToIndex()), new Procedure<ApplyLogEntries>() {
276                 @Override
277                 public void apply(ApplyLogEntries param) throws Exception {
278                 }
279             });
280
281         } else if(message instanceof ApplySnapshot ) {
282             Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
283
284             if(LOG.isDebugEnabled()) {
285                 LOG.debug("ApplySnapshot called on Follower Actor " +
286                         "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(),
287                     snapshot.getLastAppliedTerm()
288                 );
289             }
290             applySnapshot(ByteString.copyFrom(snapshot.getState()));
291
292             //clears the followers log, sets the snapshot index to ensure adjusted-index works
293             replicatedLog = new ReplicatedLogImpl(snapshot);
294             context.setReplicatedLog(replicatedLog);
295             context.setLastApplied(snapshot.getLastAppliedIndex());
296
297         } else if (message instanceof FindLeader) {
298             getSender().tell(
299                 new FindLeaderReply(getLeaderAddress()),
300                 getSelf()
301             );
302
303         } else if (message instanceof SaveSnapshotSuccess) {
304             SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
305             LOG.info("SaveSnapshotSuccess received for snapshot");
306
307             context.getReplicatedLog().snapshotCommit();
308
309             // TODO: Not sure if we want to be this aggressive with trimming stuff
310             trimPersistentData(success.metadata().sequenceNr());
311
312         } else if (message instanceof SaveSnapshotFailure) {
313             SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
314
315             LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString());
316             LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:");
317
318             context.getReplicatedLog().snapshotRollback();
319
320             LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." +
321                 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}",
322                 context.getReplicatedLog().getSnapshotIndex(),
323                 context.getReplicatedLog().getSnapshotTerm(),
324                 context.getReplicatedLog().size());
325
326         } else if (message instanceof AddRaftPeer){
327
328             // FIXME : Do not add raft peers like this.
329             // When adding a new Peer we have to ensure that the a majority of
330             // the peers know about the new Peer. Doing it this way may cause
331             // a situation where multiple Leaders may emerge
332             AddRaftPeer arp = (AddRaftPeer)message;
333            context.addToPeers(arp.getName(), arp.getAddress());
334
335         } else if (message instanceof RemoveRaftPeer){
336
337             RemoveRaftPeer rrp = (RemoveRaftPeer)message;
338             context.removePeer(rrp.getName());
339
340         } else if (message instanceof CaptureSnapshot) {
341             LOG.info("CaptureSnapshot received by actor");
342             CaptureSnapshot cs = (CaptureSnapshot)message;
343             captureSnapshot = cs;
344             createSnapshot();
345
346         } else if (message instanceof CaptureSnapshotReply){
347             LOG.info("CaptureSnapshotReply received by actor");
348             CaptureSnapshotReply csr = (CaptureSnapshotReply) message;
349
350             ByteString stateInBytes = csr.getSnapshot();
351             LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size());
352             handleCaptureSnapshotReply(stateInBytes);
353
354         } else {
355             if (!(message instanceof AppendEntriesMessages.AppendEntries)
356                 && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) {
357                 if(LOG.isDebugEnabled()) {
358                     LOG.debug("onReceiveCommand: message: {}", message.getClass());
359                 }
360             }
361
362             RaftActorBehavior oldBehavior = currentBehavior;
363             currentBehavior = currentBehavior.handleMessage(getSender(), message);
364
365             if(oldBehavior != currentBehavior){
366                 onStateChanged();
367             }
368
369             onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId());
370         }
371     }
372
373     public java.util.Set<String> getPeers() {
374         return context.getPeerAddresses().keySet();
375     }
376
377     protected String getReplicatedLogState() {
378         return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex()
379             + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm()
380             + ", im-mem journal size=" + context.getReplicatedLog().size();
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(ActorRef clientActor, String identifier,
393         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 {}", replicatedLogEntry);
401         }
402
403         replicatedLog
404             .appendAndPersist(clientActor, identifier, replicatedLogEntry);
405     }
406
407     protected String getId() {
408         return context.getId();
409     }
410
411     /**
412      * Derived actors can call the isLeader method to check if the current
413      * RaftActor is the Leader or not
414      *
415      * @return true it this RaftActor is a Leader false otherwise
416      */
417     protected boolean isLeader() {
418         return context.getId().equals(currentBehavior.getLeaderId());
419     }
420
421     /**
422      * Derived actor can call getLeader if they need a reference to the Leader.
423      * This would be useful for example in forwarding a request to an actor
424      * which is the leader
425      *
426      * @return A reference to the leader if known, null otherwise
427      */
428     protected ActorSelection getLeader(){
429         String leaderAddress = getLeaderAddress();
430
431         if(leaderAddress == null){
432             return null;
433         }
434
435         return context.actorSelection(leaderAddress);
436     }
437
438     /**
439      *
440      * @return the current leader's id
441      */
442     protected String getLeaderId(){
443         return currentBehavior.getLeaderId();
444     }
445
446     protected RaftState getRaftState() {
447         return currentBehavior.state();
448     }
449
450     protected ReplicatedLogEntry getLastLogEntry() {
451         return replicatedLog.last();
452     }
453
454     protected Long getCurrentTerm(){
455         return context.getTermInformation().getCurrentTerm();
456     }
457
458     protected Long getCommitIndex(){
459         return context.getCommitIndex();
460     }
461
462     protected Long getLastApplied(){
463         return context.getLastApplied();
464     }
465
466     protected RaftActorContext getRaftActorContext() {
467         return context;
468     }
469
470     /**
471      * setPeerAddress sets the address of a known peer at a later time.
472      * <p>
473      * This is to account for situations where a we know that a peer
474      * exists but we do not know an address up-front. This may also be used in
475      * situations where a known peer starts off in a different location and we
476      * need to change it's address
477      * <p>
478      * Note that if the peerId does not match the list of peers passed to
479      * this actor during construction an IllegalStateException will be thrown.
480      *
481      * @param peerId
482      * @param peerAddress
483      */
484     protected void setPeerAddress(String peerId, String peerAddress){
485         context.setPeerAddress(peerId, peerAddress);
486     }
487
488
489
490     /**
491      * The applyState method will be called by the RaftActor when some data
492      * needs to be applied to the actor's state
493      *
494      * @param clientActor A reference to the client who sent this message. This
495      *                    is the same reference that was passed to persistData
496      *                    by the derived actor. clientActor may be null when
497      *                    the RaftActor is behaving as a follower or during
498      *                    recovery.
499      * @param identifier  The identifier of the persisted data. This is also
500      *                    the same identifier that was passed to persistData by
501      *                    the derived actor. identifier may be null when
502      *                    the RaftActor is behaving as a follower or during
503      *                    recovery
504      * @param data        A piece of data that was persisted by the persistData call.
505      *                    This should NEVER be null.
506      */
507     protected abstract void applyState(ActorRef clientActor, String identifier,
508         Object data);
509
510     /**
511      * This method is called during recovery at the start of a batch of state entries. Derived
512      * classes should perform any initialization needed to start a batch.
513      */
514     protected abstract void startLogRecoveryBatch(int maxBatchSize);
515
516     /**
517      * This method is called during recovery to append state data to the current batch. This method
518      * is called 1 or more times after {@link #startRecoveryStateBatch}.
519      *
520      * @param data the state data
521      */
522     protected abstract void appendRecoveredLogEntry(Payload data);
523
524     /**
525      * This method is called during recovery to reconstruct the state of the actor.
526      *
527      * @param snapshot A snapshot of the state of the actor
528      */
529     protected abstract void applyRecoverySnapshot(ByteString snapshot);
530
531     /**
532      * This method is called during recovery at the end of a batch to apply the current batched
533      * log entries. This method is called after {@link #appendRecoveryLogEntry}.
534      */
535     protected abstract void applyCurrentLogRecoveryBatch();
536
537     /**
538      * This method is called when recovery is complete.
539      */
540     protected abstract void onRecoveryComplete();
541
542     /**
543      * This method will be called by the RaftActor when a snapshot needs to be
544      * created. The derived actor should respond with its current state.
545      * <p/>
546      * During recovery the state that is returned by the derived actor will
547      * be passed back to it by calling the applySnapshot  method
548      *
549      * @return The current state of the actor
550      */
551     protected abstract void createSnapshot();
552
553     /**
554      * This method can be called at any other point during normal
555      * operations when the derived actor is out of sync with it's peers
556      * and the only way to bring it in sync is by applying a snapshot
557      *
558      * @param snapshot A snapshot of the state of the actor
559      */
560     protected abstract void applySnapshot(ByteString snapshot);
561
562     /**
563      * This method will be called by the RaftActor when the state of the
564      * RaftActor changes. The derived actor can then use methods like
565      * isLeader or getLeader to do something useful
566      */
567     protected abstract void onStateChanged();
568
569     protected void onLeaderChanged(String oldLeader, String newLeader){};
570
571     private void trimPersistentData(long sequenceNumber) {
572         // Trim akka snapshots
573         // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied
574         // For now guessing that it is ANDed.
575         deleteSnapshots(new SnapshotSelectionCriteria(
576             sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
577
578         // Trim akka journal
579         deleteMessages(sequenceNumber);
580     }
581
582     private String getLeaderAddress(){
583         if(isLeader()){
584             return getSelf().path().toString();
585         }
586         String leaderId = currentBehavior.getLeaderId();
587         if (leaderId == null) {
588             return null;
589         }
590         String peerAddress = context.getPeerAddress(leaderId);
591         if(LOG.isDebugEnabled()) {
592             LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}",
593                     leaderId, peerAddress);
594         }
595
596         return peerAddress;
597     }
598
599     private void handleCaptureSnapshotReply(ByteString stateInBytes) {
600         // create a snapshot object from the state provided and save it
601         // when snapshot is saved async, SaveSnapshotSuccess is raised.
602
603         Snapshot sn = Snapshot.create(stateInBytes.toByteArray(),
604             context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1),
605             captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
606             captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
607
608         saveSnapshot(sn);
609
610         LOG.info("Persisting of snapshot done:{}", sn.getLogMessage());
611
612         //be greedy and remove entries from in-mem journal which are in the snapshot
613         // and update snapshotIndex and snapshotTerm without waiting for the success,
614
615         context.getReplicatedLog().snapshotPreCommit(stateInBytes,
616             captureSnapshot.getLastAppliedIndex(),
617             captureSnapshot.getLastAppliedTerm());
618
619         LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
620             "and term:{}", captureSnapshot.getLastAppliedIndex(),
621             captureSnapshot.getLastAppliedTerm());
622
623         captureSnapshot = null;
624         hasSnapshotCaptureInitiated = false;
625     }
626
627
628     private class ReplicatedLogImpl extends AbstractReplicatedLogImpl {
629
630         public ReplicatedLogImpl(Snapshot snapshot) {
631             super(ByteString.copyFrom(snapshot.getState()),
632                 snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(),
633                 snapshot.getUnAppliedEntries());
634         }
635
636         public ReplicatedLogImpl() {
637             super();
638         }
639
640         @Override public void removeFromAndPersist(long logEntryIndex) {
641             int adjustedIndex = adjustedIndex(logEntryIndex);
642
643             if (adjustedIndex < 0) {
644                 return;
645             }
646
647             // FIXME: Maybe this should be done after the command is saved
648             journal.subList(adjustedIndex , journal.size()).clear();
649
650             persist(new DeleteEntries(adjustedIndex), new Procedure<DeleteEntries>(){
651
652                 @Override public void apply(DeleteEntries param)
653                     throws Exception {
654                     //FIXME : Doing nothing for now
655                 }
656             });
657         }
658
659         @Override public void appendAndPersist(
660             final ReplicatedLogEntry replicatedLogEntry) {
661             appendAndPersist(null, null, replicatedLogEntry);
662         }
663
664         public void appendAndPersist(final ActorRef clientActor,
665             final String identifier,
666             final ReplicatedLogEntry replicatedLogEntry) {
667
668             if(LOG.isDebugEnabled()) {
669                 LOG.debug("Append log entry and persist {} ", replicatedLogEntry);
670             }
671
672             // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs
673             journal.add(replicatedLogEntry);
674
675             // When persisting events with persist it is guaranteed that the
676             // persistent actor will not receive further commands between the
677             // persist call and the execution(s) of the associated event
678             // handler. This also holds for multiple persist calls in context
679             // of a single command.
680             persist(replicatedLogEntry,
681                 new Procedure<ReplicatedLogEntry>() {
682                     @Override
683                     public void apply(ReplicatedLogEntry evt) throws Exception {
684                         // when a snaphsot is being taken, captureSnapshot != null
685                         if (hasSnapshotCaptureInitiated == false &&
686                             journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) {
687
688                             LOG.info("Initiating Snapshot Capture..");
689                             long lastAppliedIndex = -1;
690                             long lastAppliedTerm = -1;
691
692                             ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied());
693                             if (lastAppliedEntry != null) {
694                                 lastAppliedIndex = lastAppliedEntry.getIndex();
695                                 lastAppliedTerm = lastAppliedEntry.getTerm();
696                             }
697
698                             if(LOG.isDebugEnabled()) {
699                                 LOG.debug("Snapshot Capture logSize: {}", journal.size());
700                                 LOG.debug("Snapshot Capture lastApplied:{} ",
701                                     context.getLastApplied());
702                                 LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex);
703                                 LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm);
704                             }
705
706                             // send a CaptureSnapshot to self to make the expensive operation async.
707                             getSelf().tell(new CaptureSnapshot(
708                                 lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm),
709                                 null);
710                             hasSnapshotCaptureInitiated = true;
711                         }
712                         // Send message for replication
713                         if (clientActor != null) {
714                             currentBehavior.handleMessage(getSelf(),
715                                 new Replicate(clientActor, identifier,
716                                     replicatedLogEntry)
717                             );
718                         }
719                     }
720                 }
721             );
722         }
723
724     }
725
726     private static class DeleteEntries implements Serializable {
727         private final int fromIndex;
728
729
730         public DeleteEntries(int fromIndex) {
731             this.fromIndex = fromIndex;
732         }
733
734         public int getFromIndex() {
735             return fromIndex;
736         }
737     }
738
739
740     private class ElectionTermImpl implements ElectionTerm {
741         /**
742          * Identifier of the actor whose election term information this is
743          */
744         private long currentTerm = 0;
745         private String votedFor = null;
746
747         @Override
748         public long getCurrentTerm() {
749             return currentTerm;
750         }
751
752         @Override
753         public String getVotedFor() {
754             return votedFor;
755         }
756
757         @Override public void update(long currentTerm, String votedFor) {
758             if(LOG.isDebugEnabled()) {
759                 LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor);
760             }
761             this.currentTerm = currentTerm;
762             this.votedFor = votedFor;
763         }
764
765         @Override
766         public void updateAndPersist(long currentTerm, String votedFor){
767             update(currentTerm, votedFor);
768             // FIXME : Maybe first persist then update the state
769             persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure<UpdateElectionTerm>(){
770
771                 @Override public void apply(UpdateElectionTerm param)
772                     throws Exception {
773
774                 }
775             });
776         }
777     }
778
779     private static class UpdateElectionTerm implements Serializable {
780         private final long currentTerm;
781         private final String votedFor;
782
783         public UpdateElectionTerm(long currentTerm, String votedFor) {
784             this.currentTerm = currentTerm;
785             this.votedFor = votedFor;
786         }
787
788         public long getCurrentTerm() {
789             return currentTerm;
790         }
791
792         public String getVotedFor() {
793             return votedFor;
794         }
795     }
796
797 }