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