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