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