Make RaftActor.initiateLeadershipTransfer() private
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 1c30fe23175b5af62ffe808d9428c3361688a4b0..945af8d47d0415200d41462cc15bb7877a113a2a 100644 (file)
@@ -1,5 +1,6 @@
 /*
  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
+ * Copyright (c) 2015 Brocade Communications Systems, Inc. and others.  All rights reserved.
  *
  * This program and the accompanying materials are made available under the
  * terms of the Eclipse Public License v1.0 which accompanies this distribution,
@@ -10,23 +11,20 @@ package org.opendaylight.controller.cluster.raft;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.PoisonPill;
 import akka.japi.Procedure;
-import akka.persistence.RecoveryCompleted;
-import akka.persistence.SaveSnapshotFailure;
-import akka.persistence.SaveSnapshotSuccess;
-import akka.persistence.SnapshotOffer;
-import akka.persistence.SnapshotSelectionCriteria;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableMap;
+import com.google.common.base.Supplier;
 import com.google.common.collect.Lists;
 import java.io.Serializable;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import javax.annotation.Nonnull;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
@@ -36,14 +34,13 @@ import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersisten
 import org.opendaylight.controller.cluster.notifications.LeaderStateChanged;
 import org.opendaylight.controller.cluster.notifications.RoleChanged;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries;
-import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
-import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
-import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply;
+import org.opendaylight.controller.cluster.raft.base.messages.InitiateCaptureSnapshot;
+import org.opendaylight.controller.cluster.raft.base.messages.LeaderTransitioning;
 import org.opendaylight.controller.cluster.raft.base.messages.Replicate;
+import org.opendaylight.controller.cluster.raft.base.messages.SwitchBehavior;
 import org.opendaylight.controller.cluster.raft.behaviors.AbstractLeader;
-import org.opendaylight.controller.cluster.raft.behaviors.DelegatingRaftActorBehavior;
+import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.behaviors.Follower;
 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
 import org.opendaylight.controller.cluster.raft.client.messages.FindLeader;
@@ -51,6 +48,7 @@ import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply;
 import org.opendaylight.controller.cluster.raft.client.messages.FollowerInfo;
 import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftState;
 import org.opendaylight.controller.cluster.raft.client.messages.OnDemandRaftState;
+import org.opendaylight.controller.cluster.raft.client.messages.Shutdown;
 import org.opendaylight.controller.cluster.raft.protobuff.client.messages.Payload;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -99,51 +97,46 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis
 
-    private static final String COMMIT_SNAPSHOT = "commit_snapshot";
-
     protected final Logger LOG = LoggerFactory.getLogger(getClass());
 
-    /**
-     * The current state determines the current behavior of a RaftActor
-     * A Raft Actor always starts off in the Follower State
-     */
-    private final DelegatingRaftActorBehavior currentBehavior = new DelegatingRaftActorBehavior();
-
     /**
      * This context should NOT be passed directly to any other actor it is
      * only to be consumed by the RaftActorBehaviors
      */
     private final RaftActorContextImpl context;
 
-    private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null);
+    private final DelegatingPersistentDataProvider delegatingPersistenceProvider;
 
-    private final Procedure<Void> createSnapshotProcedure = new CreateSnapshotProcedure();
+    private final PersistentDataProvider persistentProvider;
 
-    private Stopwatch recoveryTimer;
+    private RaftActorRecoverySupport raftRecovery;
 
-    private int currentRecoveryBatchCount;
+    private RaftActorSnapshotMessageSupport snapshotSupport;
 
     private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
 
-    public RaftActor(String id, Map<String, String> peerAddresses) {
-        this(id, peerAddresses, Optional.<ConfigParams>absent());
-    }
+    private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier();
+
+    private RaftActorServerConfigurationSupport serverConfigurationSupport;
+
+    private RaftActorLeadershipTransferCohort leadershipTransferInProgress;
+
+    private boolean shuttingDown;
 
     public RaftActor(String id, Map<String, String> peerAddresses,
-         Optional<ConfigParams> configParams) {
+         Optional<ConfigParams> configParams, short payloadVersion) {
+
+        persistentProvider = new PersistentDataProvider(this);
+        delegatingPersistenceProvider = new RaftActorDelegatingPersistentDataProvider(null, persistentProvider);
 
         context = new RaftActorContextImpl(this.getSelf(),
-            this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG),
+            this.getContext(), id, new ElectionTermImpl(persistentProvider, id, LOG),
             -1, -1, peerAddresses,
-            (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), LOG);
+            (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()),
+            delegatingPersistenceProvider, LOG);
 
-        context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, delegatingPersistenceProvider, currentBehavior));
-    }
-
-    private void initRecoveryTimer() {
-        if(recoveryTimer == null) {
-            recoveryTimer = Stopwatch.createStarted();
-        }
+        context.setPayloadVersion(payloadVersion);
+        context.setReplicatedLog(ReplicatedLogImpl.newInstance(context));
     }
 
     @Override
@@ -152,165 +145,74 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 context.getConfigParams().getJournalRecoveryLogBatchSize());
 
         super.preStart();
+
+        snapshotSupport = newRaftActorSnapshotMessageSupport();
+        serverConfigurationSupport = new RaftActorServerConfigurationSupport(this);
     }
 
     @Override
     public void postStop() {
-        if(currentBehavior != null) {
-            try {
-                currentBehavior.close();
-            } catch (Exception e) {
-                LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state());
-            }
-        }
-
+        context.close();
         super.postStop();
     }
 
     @Override
-    public void handleRecover(Object message) {
-        if(persistence().isRecoveryApplicable()) {
-            if (message instanceof SnapshotOffer) {
-                onRecoveredSnapshot((SnapshotOffer) message);
-            } else if (message instanceof ReplicatedLogEntry) {
-                onRecoveredJournalLogEntry((ReplicatedLogEntry) message);
-            } else if (message instanceof ApplyLogEntries) {
-                // Handle this message for backwards compatibility with pre-Lithium versions.
-                onRecoveredApplyLogEntries(((ApplyLogEntries) message).getToIndex());
-            } else if (message instanceof ApplyJournalEntries) {
-                onRecoveredApplyLogEntries(((ApplyJournalEntries) message).getToIndex());
-            } else if (message instanceof DeleteEntries) {
-                replicatedLog().removeFrom(((DeleteEntries) message).getFromIndex());
-            } else if (message instanceof UpdateElectionTerm) {
-                context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(),
-                        ((UpdateElectionTerm) message).getVotedFor());
-            } else if (message instanceof RecoveryCompleted) {
-                onRecoveryCompletedMessage();
-            }
-        } else {
-            if (message instanceof RecoveryCompleted) {
-                // Delete all the messages from the akka journal so that we do not end up with consistency issues
-                // Note I am not using the dataPersistenceProvider and directly using the akka api here
-                deleteMessages(lastSequenceNr());
-
-                // Delete all the akka snapshots as they will not be needed
-                deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue()));
-
-                onRecoveryComplete();
-
-                initializeBehavior();
-            }
+    protected void handleRecover(Object message) {
+        if(raftRecovery == null) {
+            raftRecovery = newRaftActorRecoverySupport();
         }
-    }
 
-    private void onRecoveredSnapshot(SnapshotOffer offer) {
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: SnapshotOffer called..", persistenceId());
-        }
+        boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider);
+        if(recoveryComplete) {
+            onRecoveryComplete();
 
-        initRecoveryTimer();
+            initializeBehavior();
 
-        Snapshot snapshot = (Snapshot) offer.snapshot();
+            raftRecovery = null;
 
-        // Create a replicated log with the snapshot information
-        // The replicated log can be used later on to retrieve this snapshot
-        // when we need to install it on a peer
-
-        context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
-                currentBehavior));
-        context.setLastApplied(snapshot.getLastAppliedIndex());
-        context.setCommitIndex(snapshot.getLastAppliedIndex());
-
-        Stopwatch timer = Stopwatch.createStarted();
-
-        // Apply the snapshot to the actors state
-        applyRecoverySnapshot(snapshot.getState());
-
-        timer.stop();
-        LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" +
-                replicatedLog().size(), persistenceId(), timer.toString(),
-                replicatedLog().getSnapshotIndex(), replicatedLog().getSnapshotTerm());
-    }
-
-    private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) {
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex());
-        }
-
-        replicatedLog().append(logEntry);
-    }
-
-    private void onRecoveredApplyLogEntries(long toIndex) {
-        if(LOG.isDebugEnabled()) {
-            LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}",
-                    persistenceId(), context.getLastApplied() + 1, toIndex);
-        }
-
-        for (long i = context.getLastApplied() + 1; i <= toIndex; i++) {
-            batchRecoveredLogEntry(replicatedLog().get(i));
-        }
-
-        context.setLastApplied(toIndex);
-        context.setCommitIndex(toIndex);
-    }
-
-    private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) {
-        initRecoveryTimer();
-
-        int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize();
-        if(currentRecoveryBatchCount == 0) {
-            startLogRecoveryBatch(batchSize);
-        }
-
-        appendRecoveredLogEntry(logEntry.getData());
-
-        if(++currentRecoveryBatchCount >= batchSize) {
-            endCurrentLogRecoveryBatch();
+            if (context.getReplicatedLog().size() > 0) {
+                self().tell(new InitiateCaptureSnapshot(), self());
+                LOG.info("{}: Snapshot capture initiated after recovery", persistenceId());
+            } else {
+                LOG.info("{}: Snapshot capture NOT initiated after recovery, journal empty", persistenceId());
+            }
         }
     }
 
-    private void endCurrentLogRecoveryBatch() {
-        applyCurrentLogRecoveryBatch();
-        currentRecoveryBatchCount = 0;
-    }
-
-    private void onRecoveryCompletedMessage() {
-        if(currentRecoveryBatchCount > 0) {
-            endCurrentLogRecoveryBatch();
-        }
-
-        onRecoveryComplete();
-
-        String recoveryTime = "";
-        if(recoveryTimer != null) {
-            recoveryTimer.stop();
-            recoveryTime = " in " + recoveryTimer.toString();
-            recoveryTimer = null;
-        }
-
-        LOG.info(
-            "Recovery completed" + recoveryTime + " - Switching actor to Follower - " +
-                "Persistence Id =  " + persistenceId() +
-                " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " +
-                "journal-size={}",
-            replicatedLog().lastIndex(), replicatedLog().getSnapshotIndex(),
-            replicatedLog().getSnapshotTerm(), replicatedLog().size());
-
-        initializeBehavior();
+    protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
+        return new RaftActorRecoverySupport(context, getRaftActorRecoveryCohort());
     }
 
-    protected void initializeBehavior(){
+    @VisibleForTesting
+    void initializeBehavior(){
         changeCurrentBehavior(new Follower(context));
     }
 
+    @VisibleForTesting
     protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
+        if(getCurrentBehavior() != null) {
+            try {
+                getCurrentBehavior().close();
+            } catch(Exception e) {
+                LOG.warn("{}: Error closing behavior {}", persistence(), getCurrentBehavior(), e);
+            }
+        }
+
         reusableBehaviorStateHolder.init(getCurrentBehavior());
         setCurrentBehavior(newBehavior);
         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
     }
 
-    @Override public void handleCommand(Object message) {
-        if (message instanceof ApplyState){
+    @Override
+    protected void handleCommand(final Object message) {
+        if (serverConfigurationSupport.handleMessage(message, getSender())) {
+            return;
+        }
+        if (snapshotSupport.handleSnapshotMessage(message, getSender())) {
+            return;
+        }
+
+        if (message instanceof ApplyState) {
             ApplyState applyState = (ApplyState) message;
 
             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
@@ -328,76 +230,155 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             applyState(applyState.getClientActor(), applyState.getIdentifier(),
                 applyState.getReplicatedLogEntry().getData());
 
-        } else if (message instanceof ApplyJournalEntries){
-            ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
-            if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
-            }
+            if (!hasFollowers()) {
+                // for single node, the capture should happen after the apply state
+                // as we delete messages from the persistent journal which have made it to the snapshot
+                // capturing the snapshot before applying makes the persistent journal and snapshot out of sync
+                // and recovery shows data missing
+                context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
 
-            persistence().persist(applyEntries, NoopProcedure.instance());
-
-        } else if(message instanceof ApplySnapshot ) {
-            Snapshot snapshot = ((ApplySnapshot) message).getSnapshot();
+                context.getSnapshotManager().trimLog(context.getLastApplied());
+            }
 
+        } else if (message instanceof ApplyJournalEntries) {
+            ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
             if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: ApplySnapshot called on Follower Actor " +
-                        "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(),
-                    snapshot.getLastAppliedTerm()
-                );
+                LOG.debug("{}: Persisting ApplyJournalEntries with index={}", persistenceId(), applyEntries.getToIndex());
             }
 
-            applySnapshot(snapshot.getState());
-
-            //clears the followers log, sets the snapshot index to ensure adjusted-index works
-            context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, delegatingPersistenceProvider,
-                    currentBehavior));
-            context.setLastApplied(snapshot.getLastAppliedIndex());
+            persistence().persist(applyEntries, NoopProcedure.instance());
 
         } else if (message instanceof FindLeader) {
             getSender().tell(
                 new FindLeaderReply(getLeaderAddress()),
                 getSelf()
             );
+        } else if(message instanceof GetOnDemandRaftState) {
+            onGetOnDemandRaftStats();
+        } else if(message instanceof InitiateCaptureSnapshot) {
+            captureSnapshot();
+        } else if(message instanceof SwitchBehavior) {
+            switchBehavior(((SwitchBehavior) message));
+        } else if(message instanceof LeaderTransitioning) {
+            onLeaderTransitioning();
+        } else if(message instanceof Shutdown) {
+            onShutDown();
+        } else if(message instanceof Runnable) {
+            ((Runnable)message).run();
+        } else {
+            switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
+        }
+    }
 
-        } else if (message instanceof SaveSnapshotSuccess) {
-            SaveSnapshotSuccess success = (SaveSnapshotSuccess) message;
-            LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId());
+    private void initiateLeadershipTransfer(final RaftActorLeadershipTransferCohort.OnComplete onComplete) {
+        LOG.debug("{}: Initiating leader transfer", persistenceId());
 
-            long sequenceNumber = success.metadata().sequenceNr();
+        if(leadershipTransferInProgress == null) {
+            leadershipTransferInProgress = new RaftActorLeadershipTransferCohort(this, getSender());
+            leadershipTransferInProgress.addOnComplete(new RaftActorLeadershipTransferCohort.OnComplete() {
+                @Override
+                public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) {
+                    leadershipTransferInProgress = null;
+                }
 
-            commitSnapshot(sequenceNumber);
+                @Override
+                public void onFailure(ActorRef raftActorRef, ActorRef replyTo) {
+                    leadershipTransferInProgress = null;
+                }
+            });
 
-        } else if (message instanceof SaveSnapshotFailure) {
-            SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message;
+            leadershipTransferInProgress.addOnComplete(onComplete);
+            leadershipTransferInProgress.init();
+        } else {
+            LOG.debug("{}: prior leader transfer in progress - adding callback", persistenceId());
+            leadershipTransferInProgress.addOnComplete(onComplete);
+        }
+    }
 
-            LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:",
-                    persistenceId(), saveSnapshotFailure.cause());
+    private void onShutDown() {
+        LOG.debug("{}: onShutDown", persistenceId());
 
-            context.getSnapshotManager().rollback();
+        if(shuttingDown) {
+            return;
+        }
+
+        shuttingDown = true;
 
-        } else if (message instanceof CaptureSnapshot) {
-            LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message);
+        final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
+        if(currentBehavior.state() == RaftState.Leader && context.hasFollowers()) {
+            initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
+                @Override
+                public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) {
+                    LOG.debug("{}: leader transfer succeeded - sending PoisonPill", persistenceId());
+                    raftActorRef.tell(PoisonPill.getInstance(), raftActorRef);
+                }
 
-            context.getSnapshotManager().create(createSnapshotProcedure);
+                @Override
+                public void onFailure(ActorRef raftActorRef, ActorRef replyTo) {
+                    LOG.debug("{}: leader transfer failed - sending PoisonPill", persistenceId());
+                    raftActorRef.tell(PoisonPill.getInstance(), raftActorRef);
+                }
+            });
+        } else if(currentBehavior.state() == RaftState.Leader) {
+            pauseLeader(new TimedRunnable(context.getConfigParams().getElectionTimeOutInterval(), this) {
+                @Override
+                protected void doRun() {
+                    self().tell(PoisonPill.getInstance(), self());
+                }
 
-        } else if (message instanceof CaptureSnapshotReply) {
-            handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot());
-        } else if(message instanceof GetOnDemandRaftState) {
-            onGetOnDemandRaftStats();
-        } else if (message.equals(COMMIT_SNAPSHOT)) {
-            commitSnapshot(-1);
+                @Override
+                protected void doCancel() {
+                    self().tell(PoisonPill.getInstance(), self());
+                }
+            });
         } else {
-            reusableBehaviorStateHolder.init(getCurrentBehavior());
+            self().tell(PoisonPill.getInstance(), self());
+        }
+    }
 
-            setCurrentBehavior(currentBehavior.handleMessage(getSender(), message));
+    private void onLeaderTransitioning() {
+        LOG.debug("{}: onLeaderTransitioning", persistenceId());
+        Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
+        if(getRaftState() == RaftState.Follower && roleChangeNotifier.isPresent()) {
+            roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), null,
+                getCurrentBehavior().getLeaderPayloadVersion()), getSelf());
+        }
+    }
 
-            handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
+    private void switchBehavior(SwitchBehavior message) {
+        if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
+            RaftState newState = message.getNewState();
+            if( newState == RaftState.Leader || newState == RaftState.Follower) {
+                switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
+                getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), "");
+            } else {
+                LOG.warn("Switching to behavior : {} - not supported", newState);
+            }
         }
     }
 
+    private void switchBehavior(Supplier<RaftActorBehavior> supplier){
+        reusableBehaviorStateHolder.init(getCurrentBehavior());
+
+        setCurrentBehavior(supplier.get());
+
+        handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
+    }
+
+    @VisibleForTesting
+    RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
+        return new RaftActorSnapshotMessageSupport(context, getRaftActorSnapshotCohort());
+    }
+
     private void onGetOnDemandRaftStats() {
         // Debugging message to retrieve raft stats.
 
+        Map<String, String> peerAddresses = new HashMap<>();
+        for(String peerId: context.getPeerIds()) {
+            peerAddresses.put(peerId, context.getPeerAddress(peerId));
+        }
+
+        final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
                 .commitIndex(context.getCommitIndex())
                 .currentTerm(context.getTermInformation().getCurrentTerm())
@@ -413,9 +394,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 .snapshotIndex(replicatedLog().getSnapshotIndex())
                 .snapshotTerm(replicatedLog().getSnapshotTerm())
                 .votedFor(context.getTermInformation().getVotedFor())
-                .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses()));
+                .peerAddresses(peerAddresses)
+                .customRaftPolicyClassName(context.getConfigParams().getCustomRaftPolicyImplementationClass());
 
-        ReplicatedLogEntry lastLogEntry = getLastLogEntry();
+        ReplicatedLogEntry lastLogEntry = replicatedLog().last();
         if (lastLogEntry != null) {
             builder.lastLogIndex(lastLogEntry.getIndex());
             builder.lastLogTerm(lastLogEntry.getTerm());
@@ -445,17 +427,25 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             onStateChanged();
         }
 
-        String oldBehaviorLeaderId = oldBehavior == null ? null : oldBehaviorState.getLeaderId();
+        String lastValidLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastValidLeaderId();
         String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name();
 
         // it can happen that the state has not changed but the leader has changed.
         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
-        if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) {
+        if(!Objects.equal(lastValidLeaderId, currentBehavior.getLeaderId()) ||
+           oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
             if(roleChangeNotifier.isPresent()) {
-                roleChangeNotifier.get().tell(new LeaderStateChanged(getId(), currentBehavior.getLeaderId()), getSelf());
+                roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
+                        currentBehavior.getLeaderPayloadVersion()), getSelf());
+            }
+
+            onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId());
+
+            if(leadershipTransferInProgress != null) {
+                leadershipTransferInProgress.onNewLeader(currentBehavior.getLeaderId());
             }
 
-            onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId());
+            serverConfigurationSupport.onNewLeader(currentBehavior.getLeaderId());
         }
 
         if (roleChangeNotifier.isPresent() &&
@@ -465,6 +455,20 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
+    protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId, short leaderPayloadVersion) {
+        return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion);
+    }
+
+    @Override
+    public long snapshotSequenceNr() {
+        // When we do a snapshot capture, we also capture and save the sequence-number of the persistent journal,
+        // so that we can delete the persistent journal based on the saved sequence-number
+        // However , when akka replays the journal during recovery, it replays it from the sequence number when the snapshot
+        // was saved and not the number we saved.
+        // We would want to override it , by asking akka to use the last-sequence number known to us.
+        return context.getSnapshotManager().getLastSequenceNumber();
+    }
+
     /**
      * When a derived RaftActor needs to persist something it must call
      * persistData.
@@ -488,24 +492,24 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
         replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
             @Override
-            public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception {
-                if(!hasFollowers()){
+            public void apply(ReplicatedLogEntry replicatedLogEntry) {
+                if (!hasFollowers()){
                     // Increment the Commit Index and the Last Applied values
                     raftContext.setCommitIndex(replicatedLogEntry.getIndex());
                     raftContext.setLastApplied(replicatedLogEntry.getIndex());
 
-                    // Apply the state immediately
-                    applyState(clientActor, identifier, data);
+                    // Apply the state immediately.
+                    self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), self());
 
                     // Send a ApplyJournalEntries message so that we write the fact that we applied
                     // the state to durable storage
                     self().tell(new ApplyJournalEntries(replicatedLogEntry.getIndex()), self());
 
-                    context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
-
                 } else if (clientActor != null) {
+                    context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
+
                     // Send message for replication
-                    currentBehavior.handleMessage(getSelf(),
+                    getCurrentBehavior().handleMessage(getSelf(),
                             new Replicate(clientActor, identifier, replicatedLogEntry));
                 }
             }
@@ -522,11 +526,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     @VisibleForTesting
     void setCurrentBehavior(RaftActorBehavior behavior) {
-        currentBehavior.setDelegate(behavior);
+        context.setCurrentBehavior(behavior);
     }
 
     protected RaftActorBehavior getCurrentBehavior() {
-        return currentBehavior.getDelegate();
+        return context.getCurrentBehavior();
     }
 
     /**
@@ -536,7 +540,16 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      * @return true it this RaftActor is a Leader false otherwise
      */
     protected boolean isLeader() {
-        return context.getId().equals(currentBehavior.getLeaderId());
+        return context.getId().equals(getCurrentBehavior().getLeaderId());
+    }
+
+    protected final boolean isLeaderActive() {
+        return getRaftState() != RaftState.IsolatedLeader && !shuttingDown &&
+                !isLeadershipTransferInProgress();
+    }
+
+    private boolean isLeadershipTransferInProgress() {
+        return leadershipTransferInProgress != null && leadershipTransferInProgress.isTransferring();
     }
 
     /**
@@ -560,36 +573,50 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      *
      * @return the current leader's id
      */
-    protected String getLeaderId(){
-        return currentBehavior.getLeaderId();
-    }
-
-    protected RaftState getRaftState() {
-        return currentBehavior.state();
+    protected final String getLeaderId(){
+        return getCurrentBehavior().getLeaderId();
     }
 
-    protected ReplicatedLogEntry getLastLogEntry() {
-        return replicatedLog().last();
+    @VisibleForTesting
+    protected final RaftState getRaftState() {
+        return getCurrentBehavior().state();
     }
 
     protected Long getCurrentTerm(){
         return context.getTermInformation().getCurrentTerm();
     }
 
-    protected Long getCommitIndex(){
-        return context.getCommitIndex();
-    }
-
-    protected Long getLastApplied(){
-        return context.getLastApplied();
-    }
-
     protected RaftActorContext getRaftActorContext() {
         return context;
     }
 
     protected void updateConfigParams(ConfigParams configParams) {
+
+        // obtain the RaftPolicy for oldConfigParams and the updated one.
+        String oldRaftPolicy = context.getConfigParams().
+            getCustomRaftPolicyImplementationClass();
+        String newRaftPolicy = configParams.
+            getCustomRaftPolicyImplementationClass();
+
+        LOG.debug("{}: RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}", persistenceId(),
+            oldRaftPolicy, newRaftPolicy);
         context.setConfigParams(configParams);
+        if (!Objects.equal(oldRaftPolicy, newRaftPolicy)) {
+            // The RaftPolicy was modified. If the current behavior is Follower then re-initialize to Follower
+            // but transfer the previous leaderId so it doesn't immediately try to schedule an election. This
+            // avoids potential disruption. Otherwise, switch to Follower normally.
+            RaftActorBehavior behavior = getCurrentBehavior();
+            if(behavior instanceof Follower) {
+                String previousLeaderId = ((Follower)behavior).getLeaderId();
+                short previousLeaderPayloadVersion = behavior.getLeaderPayloadVersion();
+
+                LOG.debug("{}: Re-initializing to Follower with previous leaderId {}", persistenceId(), previousLeaderId);
+
+                changeCurrentBehavior(new Follower(context, previousLeaderId, previousLeaderPayloadVersion));
+            } else {
+                initializeBehavior();
+            }
+        }
     }
 
     public final DataPersistenceProvider persistence() {
@@ -621,7 +648,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                     // Make saving Snapshot successful
                     // Committing the snapshot here would end up calling commit in the creating state which would
                     // be a state violation. That's why now we send a message to commit the snapshot.
-                    self().tell(COMMIT_SNAPSHOT, self());
+                    self().tell(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self());
                 }
             });
         }
@@ -645,10 +672,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         context.setPeerAddress(peerId, peerAddress);
     }
 
-    protected void commitSnapshot(long sequenceNumber) {
-        context.getSnapshotManager().commit(persistence(), sequenceNumber);
-    }
-
     /**
      * The applyState method will be called by the RaftActor when some data
      * needs to be applied to the actor's state
@@ -670,31 +693,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         Object data);
 
     /**
-     * This method is called during recovery at the start of a batch of state entries. Derived
-     * classes should perform any initialization needed to start a batch.
-     */
-    protected abstract void startLogRecoveryBatch(int maxBatchSize);
-
-    /**
-     * This method is called during recovery to append state data to the current batch. This method
-     * is called 1 or more times after {@link #startLogRecoveryBatch}.
-     *
-     * @param data the state data
-     */
-    protected abstract void appendRecoveredLogEntry(Payload data);
-
-    /**
-     * This method is called during recovery to reconstruct the state of the actor.
-     *
-     * @param snapshotBytes A snapshot of the state of the actor
-     */
-    protected abstract void applyRecoverySnapshot(byte[] snapshotBytes);
-
-    /**
-     * This method is called during recovery at the end of a batch to apply the current batched
-     * log entries. This method is called after {@link #appendRecoveredLogEntry}.
+     * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
      */
-    protected abstract void applyCurrentLogRecoveryBatch();
+    @Nonnull
+    protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort();
 
     /**
      * This method is called when recovery is complete.
@@ -702,24 +704,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     protected abstract void onRecoveryComplete();
 
     /**
-     * This method will be called by the RaftActor when a snapshot needs to be
-     * created. The derived actor should respond with its current state.
-     * <p/>
-     * During recovery the state that is returned by the derived actor will
-     * be passed back to it by calling the applySnapshot  method
-     *
-     * @return The current state of the actor
+     * Returns the RaftActorSnapshotCohort to participate in persistence recovery.
      */
-    protected abstract void createSnapshot();
-
-    /**
-     * This method can be called at any other point during normal
-     * operations when the derived actor is out of sync with it's peers
-     * and the only way to bring it in sync is by applying a snapshot
-     *
-     * @param snapshotBytes A snapshot of the state of the actor
-     */
-    protected abstract void applySnapshot(byte[] snapshotBytes);
+    @Nonnull
+    protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort();
 
     /**
      * This method will be called by the RaftActor when the state of the
@@ -734,13 +722,30 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      */
     protected abstract Optional<ActorRef> getRoleChangeNotifier();
 
-    protected void onLeaderChanged(String oldLeader, String newLeader){};
+    /**
+     * This method is called prior to operations such as leadership transfer and actor shutdown when the leader
+     * must pause or stop its duties. This method allows derived classes to gracefully pause or finish current
+     * work prior to performing the operation. On completion of any work, the run method must be called on the
+     * given Runnable to proceed with the given operation. <b>Important:</b> the run method must be called on
+     * this actor's thread dispatcher as as it modifies internal state.
+     * <p>
+     * The default implementation immediately runs the operation.
+     *
+     * @param operation the operation to run
+     */
+    protected void pauseLeader(Runnable operation) {
+        operation.run();
+    }
+
+    protected void onLeaderChanged(String oldLeader, String newLeader) {
+
+    };
 
     private String getLeaderAddress(){
         if(isLeader()){
             return getSelf().path().toString();
         }
-        String leaderId = currentBehavior.getLeaderId();
+        String leaderId = getLeaderId();
         if (leaderId == null) {
             return null;
         }
@@ -753,18 +758,61 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         return peerAddress;
     }
 
-    private void handleCaptureSnapshotReply(byte[] snapshotBytes) {
-        LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length);
+    protected boolean hasFollowers(){
+        return getRaftActorContext().hasFollowers();
+    }
+
+    private void captureSnapshot() {
+        SnapshotManager snapshotManager = context.getSnapshotManager();
+
+        if (!snapshotManager.isCapturing()) {
+            final long idx = getCurrentBehavior().getReplicatedToAllIndex();
+            LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}",
+                replicatedLog().last(), idx);
 
-        context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory());
+            snapshotManager.capture(replicatedLog().last(), idx);
+        }
     }
 
-    protected boolean hasFollowers(){
-        return getRaftActorContext().hasFollowers();
+    /**
+     * Switch this member to non-voting status. This is a no-op for all behaviors except when we are the leader,
+     * in which case we need to step down.
+     */
+    void becomeNonVoting() {
+        if (isLeader()) {
+            initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() {
+                @Override
+                public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) {
+                    LOG.debug("{}: leader transfer succeeded after change to non-voting", persistenceId());
+                    ensureFollowerState();
+                }
+
+                @Override
+                public void onFailure(ActorRef raftActorRef, ActorRef replyTo) {
+                    LOG.debug("{}: leader transfer failed after change to non-voting", persistenceId());
+                    ensureFollowerState();
+                }
+
+                private void ensureFollowerState() {
+                    // Whether or not leadership transfer succeeded, we have to step down as leader and
+                    // switch to Follower so ensure that.
+                    if (getRaftState() != RaftState.Follower) {
+                        initializeBehavior();
+                    }
+                }
+            });
+        }
     }
 
+    /**
+     * @deprecated Deprecated in favor of {@link org.opendaylight.controller.cluster.raft.base.messages.DeleteEntries}
+     *             whose type for fromIndex is long instead of int. This class was kept for backwards
+     *             compatibility with Helium.
+     */
+    // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
+    @SuppressWarnings("serial")
+    @Deprecated
     static class DeleteEntries implements Serializable {
-        private static final long serialVersionUID = 1L;
         private final int fromIndex;
 
         public DeleteEntries(int fromIndex) {
@@ -776,8 +824,14 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
+    /**
+     * @deprecated Deprecated in favor of non-inner class {@link org.opendaylight.controller.cluster.raft.base.messages.UpdateElectionTerm}
+     *             which has serialVersionUID set. This class was kept for backwards compatibility with Helium.
+     */
+    // Suppressing this warning as we can't set serialVersionUID to maintain backwards compatibility.
+    @SuppressWarnings("serial")
+    @Deprecated
     static class UpdateElectionTerm implements Serializable {
-        private static final long serialVersionUID = 1L;
         private final long currentTerm;
         private final String votedFor;
 
@@ -795,29 +849,50 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-    private class CreateSnapshotProcedure implements Procedure<Void> {
-
-        @Override
-        public void apply(Void aVoid) throws Exception {
-            createSnapshot();
-        }
-    }
-
     private static class BehaviorStateHolder {
         private RaftActorBehavior behavior;
-        private String leaderId;
+        private String lastValidLeaderId;
+        private short leaderPayloadVersion;
 
         void init(RaftActorBehavior behavior) {
             this.behavior = behavior;
-            this.leaderId = behavior != null ? behavior.getLeaderId() : null;
+            this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1;
+
+            String behaviorLeaderId = behavior != null ? behavior.getLeaderId() : null;
+            if(behaviorLeaderId != null) {
+                this.lastValidLeaderId = behaviorLeaderId;
+            }
         }
 
         RaftActorBehavior getBehavior() {
             return behavior;
         }
 
-        String getLeaderId() {
-            return leaderId;
+        String getLastValidLeaderId() {
+            return lastValidLeaderId;
+        }
+
+        short getLeaderPayloadVersion() {
+            return leaderPayloadVersion;
+        }
+    }
+
+    private class SwitchBehaviorSupplier implements Supplier<RaftActorBehavior> {
+        private Object message;
+        private ActorRef sender;
+
+        public SwitchBehaviorSupplier handleMessage(ActorRef sender, Object message){
+            this.sender = sender;
+            this.message = message;
+            return this;
+        }
+
+        @Override
+        public RaftActorBehavior get() {
+            if(this.message instanceof SwitchBehavior){
+                return AbstractRaftActorBehavior.createBehavior(context, ((SwitchBehavior) message).getNewState());
+            }
+            return getCurrentBehavior().handleMessage(sender, message);
         }
     }
 }