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