Bug 3020: Add leader version to LeaderStateChanged
[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  * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
4  *
5  * This program and the accompanying materials are made available under the
6  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
7  * and is available at http://www.eclipse.org/legal/epl-v10.html
8  */
9
10 package org.opendaylight.controller.cluster.raft;
11
12 import akka.actor.ActorRef;
13 import akka.actor.ActorSelection;
14 import akka.japi.Procedure;
15 import akka.persistence.SnapshotSelectionCriteria;
16 import com.google.common.annotations.VisibleForTesting;
17 import com.google.common.base.Objects;
18 import com.google.common.base.Optional;
19 import com.google.common.collect.Lists;
20 import java.io.Serializable;
21 import java.util.Collection;
22 import java.util.HashMap;
23 import java.util.List;
24 import java.util.Map;
25 import java.util.concurrent.TimeUnit;
26 import javax.annotation.Nonnull;
27 import org.apache.commons.lang3.time.DurationFormatUtils;
28 import org.opendaylight.controller.cluster.DataPersistenceProvider;
29 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
30 import org.opendaylight.controller.cluster.NonPersistentDataProvider;
31 import org.opendaylight.controller.cluster.PersistentDataProvider;
32 import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor;
33 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
34 import org.opendaylight.controller.cluster.notifications.RoleChanged;
35 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
36 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
37 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
38 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
39 import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
40 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
41 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
42 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
43 import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
44 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
45 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
46 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
47 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
48 import org.slf4j.Logger;
49 import org.slf4j.LoggerFactory;
50
51 /**
52  * RaftActor encapsulates a state machine that needs to be kept synchronized
53  * in a cluster. It implements the RAFT algorithm as described in the paper
54  * <a href='https://ramcloud.stanford.edu/wiki/download/attachments/11370504/raft.pdf'>
55  * In Search of an Understandable Consensus Algorithm</a>
56  * <p/>
57  * RaftActor has 3 states and each state has a certain behavior associated
58  * with it. A Raft actor can behave as,
59  * <ul>
60  * <li> A Leader </li>
61  * <li> A Follower (or) </li>
62  * <li> A Candidate </li>
63  * </ul>
64  * <p/>
65  * <p/>
66  * A RaftActor MUST be a Leader in order to accept requests from clients to
67  * change the state of it's encapsulated state machine. Once a RaftActor becomes
68  * a Leader it is also responsible for ensuring that all followers ultimately
69  * have the same log and therefore the same state machine as itself.
70  * <p/>
71  * <p/>
72  * The current behavior of a RaftActor determines how election for leadership
73  * is initiated and how peer RaftActors react to request for votes.
74  * <p/>
75  * <p/>
76  * Each RaftActor also needs to know the current election term. It uses this
77  * information for a couple of things. One is to simply figure out who it
78  * voted for in the last election. Another is to figure out if the message
79  * it received to update it's state is stale.
80  * <p/>
81  * <p/>
82  * The RaftActor uses akka-persistence to store it's replicated log.
83  * Furthermore through it's behaviors a Raft Actor determines
84  * <p/>
85  * <ul>
86  * <li> when a log entry should be persisted </li>
87  * <li> when a log entry should be applied to the state machine (and) </li>
88  * <li> when a snapshot should be saved </li>
89  * </ul>
90  */
91 public abstract class RaftActor extends AbstractUntypedPersistentActor {
92
93     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
94
95     protected final Logger LOG = LoggerFactory.getLogger(getClass());
96
97     /**
98      * The current state determines the current behavior of a RaftActor
99      * A Raft Actor always starts off in the Follower State
100      */
101     private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
102
103     /**
104      * This context should NOT be passed directly to any other actor it is
105      * only to be consumed by the RaftActorBehaviors
106      */
107     private final RaftActorContextImpl context;
108
109     private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
110
111     private RaftActorRecoverySupport raftRecovery;
112
113     private RaftActorSnapshotMessageSupport snapshotSupport;
114
115     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
116
117     public RaftActor(String id, Map<String, String> peerAddresses,
118          Optional<ConfigParams> configParams, short payloadVersion) {
119
120         context = new RaftActorContextImpl(this.getSelf(),
121             this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
122             -1, -1, peerAddresses,
123             (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
124             delegatingPersistenceProvider, LOG);
125
126         context.setPayloadVersion(payloadVersion);
127         context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior));
128     }
129
130     @Override
131     public void preStart() throws Exception {
132         LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(),
133                 context.getConfigParams().getJournalRecoveryLogBatchSize());
134
135         super.preStart();
136
137         snapshotSupport = newRaftActorSnapshotMessageSupport();
138     }
139
140     @Override
141     public void postStop() {
142         if(currentBehavior.getDelegate() != null) {
143             try {
144                 currentBehavior.close();
145             } catch (Exception e) {
146                 LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
147             }
148         }
149
150         super.postStop();
151     }
152
153     @Override
154     public void handleRecover(Object message) {
155         if(raftRecovery == null) {
156             raftRecovery = newRaftActorRecoverySupport();
157         }
158
159         boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message);
160         if(recoveryComplete) {
161             if(!persistence().isRecoveryApplicable()) {
162                 // Delete all the messages from the akka journal so that we do not end up with consistency issues
163                 // Note I am not using the dataPersistenceProvider and directly using the akka api here
164                 deleteMessages(lastSequenceNr());
165
166                 // Delete all the akka snapshots as they will not be needed
167                 deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
168             }
169
170             onRecoveryComplete();
171
172             initializeBehavior();
173
174             raftRecovery = null;
175         }
176     }
177
178     protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
179         return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort());
180     }
181
182     protected void initializeBehavior(){
183         changeCurrentBehavior(new Follower(context));
184     }
185
186     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
187         reusableBehaviorStateHolder.init(getCurrentBehavior());
188         setCurrentBehavior(newBehavior);
189         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
190     }
191
192     @Override
193     public void handleCommand(Object message) {
194         if (message instanceof ApplyState){
195             ApplyState applyState = (ApplyState) message;
196
197             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
198             if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
199                 LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
200                         TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
201             }
202
203             if(LOG.isDebugEnabled()) {
204                 LOG.debug("{}: Applying state for log index {} data {}",
205                     persistenceId(), applyState.getReplicatedLogEntry().getIndex(),
206                     applyState.getReplicatedLogEntry().getData());
207             }
208
209             applyState(applyState.getClientActor(), applyState.getIdentifier(),
210                 applyState.getReplicatedLogEntry().getData());
211
212             if (!hasFollowers()) {
213                 // for single node, the capture should happen after the apply state
214                 // as we delete messages from the persistent journal which have made it to the snapshot
215                 // capturing the snapshot before applying makes the persistent journal and snapshot out of sync
216                 // and recovery shows data missing
217                 context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
218
219                 context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
220             }
221
222         } else if (message instanceof ApplyJournalEntries){
223             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
224             if(LOG.isDebugEnabled()) {
225                 LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
226             }
227
228             persistence().persist(applyEntries, NoopProcedure.instance());
229
230         } else if (message instanceof FindLeader) {
231             getSender().tell(
232                 new FindLeaderReply(getLeaderAddress()),
233                 getSelf()
234             );
235         } else if(message instanceof GetOnDemandRaftState) {
236             onGetOnDemandRaftStats();
237         } else if(!snapshotSupport.handleSnapshotMessage(message)) {
238             reusableBehaviorStateHolder.init(getCurrentBehavior());
239
240             setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
241
242             handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
243         }
244     }
245
246     protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
247         return new RaftActorSnapshotMessageSupport(context, currentBehavior,
248                 getRaftActorSnapshotCohort());
249     }
250
251     private void onGetOnDemandRaftStats() {
252         // Debugging message to retrieve raft stats.
253
254         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
255                 .commitIndex(context.getCommitIndex())
256                 .currentTerm(context.getTermInformation().getCurrentTerm())
257                 .inMemoryJournalDataSize(replicatedLog().dataSize())
258                 .inMemoryJournalLogSize(replicatedLog().size())
259                 .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing())
260                 .lastApplied(context.getLastApplied())
261                 .lastIndex(replicatedLog().lastIndex())
262                 .lastTerm(replicatedLog().lastTerm())
263                 .leader(getLeaderId())
264                 .raftState(currentBehavior.state().toString())
265                 .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex())
266                 .snapshotIndex(replicatedLog().getSnapshotIndex())
267                 .snapshotTerm(replicatedLog().getSnapshotTerm())
268                 .votedFor(context.getTermInformation().getVotedFor())
269                 .peerAddresses(new HashMap<>(context.getPeerAddresses()));
270
271         ReplicatedLogEntry lastLogEntry = getLastLogEntry();
272         if (lastLogEntry != null) {
273             builder.lastLogIndex(lastLogEntry.getIndex());
274             builder.lastLogTerm(lastLogEntry.getTerm());
275         }
276
277         if(getCurrentBehavior() instanceof AbstractLeader) {
278             AbstractLeader leader = (AbstractLeader)getCurrentBehavior();
279             Collection<String> followerIds = leader.getFollowerIds();
280             List<FollowerInfo> followerInfoList = Lists.newArrayListWithCapacity(followerIds.size());
281             for(String id: followerIds) {
282                 final FollowerLogInformation info = leader.getFollower(id);
283                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
284                         info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
285             }
286
287             builder.followerInfoList(followerInfoList);
288         }
289
290         sender().tell(builder.build(), self());
291
292     }
293
294     private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
295         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
296
297         if (oldBehavior != currentBehavior){
298             onStateChanged();
299         }
300
301         String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
302         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
303
304         // it can happen that the state has not changed but the leader has changed.
305         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
306         if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId()) ||
307            oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
308             if(roleChangeNotifier.isPresent()) {
309                 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
310                         currentBehavior.getLeaderPayloadVersion()), getSelf());
311             }
312
313             onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
314         }
315
316         if (roleChangeNotifier.isPresent() &&
317                 (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) {
318             roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName ,
319                     currentBehavior.state().name()), getSelf());
320         }
321     }
322
323     protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId, short leaderPayloadVersion) {
324         return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
325     }
326
327     @Override
328     public long snapshotSequenceNr() {
329         // When we do a snapshot capture, we also capture and save the sequence-number of the persistent journal,
330         // so that we can delete the persistent journal based on the saved sequence-number
331         // However , when akka replays the journal during recovery, it replays it from the sequence number when the snapshot
332         // was saved and not the number we saved.
333         // We would want to override it , by asking akka to use the last-sequence number known to us.
334         return context.getSnapshotManager().getLastSequenceNumber();
335     }
336
337     /**
338      * When a derived RaftActor needs to persist something it must call
339      * persistData.
340      *
341      * @param clientActor
342      * @param identifier
343      * @param data
344      */
345     protected void persistData(final ActorRef clientActor, final String identifier,
346         final Payload data) {
347
348         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
349             context.getReplicatedLog().lastIndex() + 1,
350             context.getTermInformation().getCurrentTerm(), data);
351
352         if(LOG.isDebugEnabled()) {
353             LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry);
354         }
355
356         final RaftActorContext raftContext = getRaftActorContext();
357
358         replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
359             @Override
360             public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
361                 if (!hasFollowers()){
362                     // Increment the Commit Index and the Last Applied values
363                     raftContext.setCommitIndex(replicatedLogEntry.getIndex());
364                     raftContext.setLastApplied(replicatedLogEntry.getIndex());
365
366                     // Apply the state immediately.
367                     self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), self());
368
369                     // Send a ApplyJournalEntries message so that we write the fact that we applied
370                     // the state to durable storage
371                     self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
372
373                 } else if (clientActor != null) {
374                     context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
375
376                     // Send message for replication
377                     currentBehavior.handleMessage(getSelf(),
378                             new Replicate(clientActor, identifier, replicatedLogEntry));
379                 }
380             }
381         });
382     }
383
384     private ReplicatedLog replicatedLog() {
385         return context.getReplicatedLog();
386     }
387
388     protected String getId() {
389         return context.getId();
390     }
391
392     @VisibleForTesting
393     void setCurrentBehavior(RaftActorBehavior behavior) {
394         currentBehavior.setDelegate(behavior);
395     }
396
397     protected RaftActorBehavior getCurrentBehavior() {
398         return currentBehavior.getDelegate();
399     }
400
401     /**
402      * Derived actors can call the isLeader method to check if the current
403      * RaftActor is the Leader or not
404      *
405      * @return true it this RaftActor is a Leader false otherwise
406      */
407     protected boolean isLeader() {
408         return context.getId().equals(currentBehavior.getLeaderId());
409     }
410
411     /**
412      * Derived actor can call getLeader if they need a reference to the Leader.
413      * This would be useful for example in forwarding a request to an actor
414      * which is the leader
415      *
416      * @return A reference to the leader if known, null otherwise
417      */
418     protected ActorSelection getLeader(){
419         String leaderAddress = getLeaderAddress();
420
421         if(leaderAddress == null){
422             return null;
423         }
424
425         return context.actorSelection(leaderAddress);
426     }
427
428     /**
429      *
430      * @return the current leader's id
431      */
432     protected String getLeaderId(){
433         return currentBehavior.getLeaderId();
434     }
435
436     protected RaftState getRaftState() {
437         return currentBehavior.state();
438     }
439
440     protected ReplicatedLogEntry getLastLogEntry() {
441         return replicatedLog().last();
442     }
443
444     protected Long getCurrentTerm(){
445         return context.getTermInformation().getCurrentTerm();
446     }
447
448     protected Long getCommitIndex(){
449         return context.getCommitIndex();
450     }
451
452     protected Long getLastApplied(){
453         return context.getLastApplied();
454     }
455
456     protected RaftActorContext getRaftActorContext() {
457         return context;
458     }
459
460     protected void updateConfigParams(ConfigParams configParams) {
461         context.setConfigParams(configParams);
462     }
463
464     public final DataPersistenceProvider persistence() {
465         return delegatingPersistenceProvider.getDelegate();
466     }
467
468     public void setPersistence(DataPersistenceProvider provider) {
469         delegatingPersistenceProvider.setDelegate(provider);
470     }
471
472     protected void setPersistence(boolean persistent) {
473         if(persistent) {
474             setPersistence(new PersistentDataProvider(this));
475         } else {
476             setPersistence(new NonPersistentDataProvider() {
477                 /**
478                  * The way snapshotting works is,
479                  * <ol>
480                  * <li> RaftActor calls createSnapshot on the Shard
481                  * <li> Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot
482                  * <li> When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save
483                  * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the
484                  * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This
485                  * commitSnapshot is mimicking what is done in SaveSnapshotSuccess.
486                  * </ol>
487                  */
488                 @Override
489                 public void saveSnapshot(Object o) {
490                     // Make saving Snapshot successful
491                     // Committing the snapshot here would end up calling commit in the creating state which would
492                     // be a state violation. That's why now we send a message to commit the snapshot.
493                     self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
494                 }
495             });
496         }
497     }
498
499     /**
500      * setPeerAddress sets the address of a known peer at a later time.
501      * <p>
502      * This is to account for situations where a we know that a peer
503      * exists but we do not know an address up-front. This may also be used in
504      * situations where a known peer starts off in a different location and we
505      * need to change it's address
506      * <p>
507      * Note that if the peerId does not match the list of peers passed to
508      * this actor during construction an IllegalStateException will be thrown.
509      *
510      * @param peerId
511      * @param peerAddress
512      */
513     protected void setPeerAddress(String peerId, String peerAddress){
514         context.setPeerAddress(peerId, peerAddress);
515     }
516
517     /**
518      * The applyState method will be called by the RaftActor when some data
519      * needs to be applied to the actor's state
520      *
521      * @param clientActor A reference to the client who sent this message. This
522      *                    is the same reference that was passed to persistData
523      *                    by the derived actor. clientActor may be null when
524      *                    the RaftActor is behaving as a follower or during
525      *                    recovery.
526      * @param identifier  The identifier of the persisted data. This is also
527      *                    the same identifier that was passed to persistData by
528      *                    the derived actor. identifier may be null when
529      *                    the RaftActor is behaving as a follower or during
530      *                    recovery
531      * @param data        A piece of data that was persisted by the persistData call.
532      *                    This should NEVER be null.
533      */
534     protected abstract void applyState(ActorRef clientActor, String identifier,
535         Object data);
536
537     /**
538      * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
539      */
540     @Nonnull
541     protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
542
543     /**
544      * This method is called when recovery is complete.
545      */
546     protected abstract void onRecoveryComplete();
547
548     /**
549      * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
550      */
551     @Nonnull
552     protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
553
554     /**
555      * This method will be called by the RaftActor when the state of the
556      * RaftActor changes. The derived actor can then use methods like
557      * isLeader or getLeader to do something useful
558      */
559     protected abstract void onStateChanged();
560
561     /**
562      * Notifier Actor for this RaftActor to notify when a role change happens
563      * @return ActorRef - ActorRef of the notifier or Optional.absent if none.
564      */
565     protected abstract Optional<ActorRef> getRoleChangeNotifier();
566
567     protected void onLeaderChanged(String oldLeader, String newLeader){};
568
569     private String getLeaderAddress(){
570         if(isLeader()){
571             return getSelf().path().toString();
572         }
573         String leaderId = currentBehavior.getLeaderId();
574         if (leaderId == null) {
575             return null;
576         }
577         String peerAddress = context.getPeerAddress(leaderId);
578         if(LOG.isDebugEnabled()) {
579             LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}",
580                     persistenceId(), leaderId, peerAddress);
581         }
582
583         return peerAddress;
584     }
585
586     protected boolean hasFollowers(){
587         return getRaftActorContext().hasFollowers();
588     }
589
590     /**
591      * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntries}
592      *             whose type for fromIndex is long instead of int. This class was kept for backwards
593      *             compatibility with Helium.
594      */
595     // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
596     @SuppressWarnings("serial")
597     @Deprecated
598     static class DeleteEntries implements Serializable {
599         private final int fromIndex;
600
601         public DeleteEntries(int fromIndex) {
602             this.fromIndex = fromIndex;
603         }
604
605         public int getFromIndex() {
606             return fromIndex;
607         }
608     }
609
610     /**
611      * @deprecated Deprecated in favor of non-inner class {@link org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm}
612      *             which has serialVersionUID set. This class was kept for backwards compatibility with Helium.
613      */
614     // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
615     @SuppressWarnings("serial")
616     @Deprecated
617     static class UpdateElectionTerm implements Serializable {
618         private final long currentTerm;
619         private final String votedFor;
620
621         public UpdateElectionTerm(long currentTerm, String votedFor) {
622             this.currentTerm = currentTerm;
623             this.votedFor = votedFor;
624         }
625
626         public long getCurrentTerm() {
627             return currentTerm;
628         }
629
630         public String getVotedFor() {
631             return votedFor;
632         }
633     }
634
635     private static class BehaviorStateHolder {
636         private RaftActorBehavior behavior;
637         private String leaderId;
638         private short leaderPayloadVersion;
639
640         void init(RaftActorBehavior behavior) {
641             this.behavior = behavior;
642             this.leaderId = behavior != null ? behavior.getLeaderId() : null;
643             this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1;
644         }
645
646         RaftActorBehavior getBehavior() {
647             return behavior;
648         }
649
650         String getLeaderId() {
651             return leaderId;
652         }
653
654         short getLeaderPayloadVersion() {
655             return leaderPayloadVersion;
656         }
657     }
658 }