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