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