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