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