Make RaftActor.initiateLeadershipTransfer() private
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 5f6f3ec24e573c4dedba6ba42a38dbbe28a51698..945af8d47d0415200d41462cc15bb7877a113a2a 100644 (file)
@@ -11,6 +11,7 @@ package org.opendaylight.controller.cluster.raft;
 
 import akka.actor.ActorRef;
 import akka.actor.ActorSelection;
+import akka.actor.PoisonPill;
 import akka.japi.Procedure;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
@@ -35,10 +36,11 @@ import org.opendaylight.controller.cluster.notifications.RoleChanged;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyJournalEntries;
 import org.opendaylight.controller.cluster.raft.base.messages.ApplyState;
 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;
@@ -46,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;
@@ -96,12 +99,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     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
@@ -122,6 +119,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     private RaftActorServerConfigurationSupport serverConfigurationSupport;
 
+    private RaftActorLeadershipTransferCohort leadershipTransferInProgress;
+
+    private boolean shuttingDown;
+
     public RaftActor(String id, Map<String, String> peerAddresses,
          Optional<ConfigParams> configParams, short payloadVersion) {
 
@@ -135,7 +136,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             delegatingPersistenceProvider, LOG);
 
         context.setPayloadVersion(payloadVersion);
-        context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior));
+        context.setReplicatedLog(ReplicatedLogImpl.newInstance(context));
     }
 
     @Override
@@ -146,24 +147,17 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         super.preStart();
 
         snapshotSupport = newRaftActorSnapshotMessageSupport();
-        serverConfigurationSupport = new RaftActorServerConfigurationSupport(getRaftActorContext());
+        serverConfigurationSupport = new RaftActorServerConfigurationSupport(this);
     }
 
     @Override
     public void postStop() {
-        if(currentBehavior.getDelegate() != 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) {
+    protected void handleRecover(Object message) {
         if(raftRecovery == null) {
             raftRecovery = newRaftActorRecoverySupport();
         }
@@ -178,32 +172,47 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
             if (context.getReplicatedLog().size() > 0) {
                 self().tell(new InitiateCaptureSnapshot(), self());
-                LOG.info("Snapshot capture initiated after recovery");
+                LOG.info("{}: Snapshot capture initiated after recovery", persistenceId());
             } else {
-                LOG.info("Snapshot capture NOT initiated after recovery, journal empty");
+                LOG.info("{}: Snapshot capture NOT initiated after recovery, journal empty", persistenceId());
             }
         }
     }
 
     protected RaftActorRecoverySupport newRaftActorRecoverySupport() {
-        return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort());
+        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(final Object message) {
-        if(serverConfigurationSupport.handleMessage(message, this, getSender())) {
+    protected void handleCommand(final Object message) {
+        if (serverConfigurationSupport.handleMessage(message, getSender())) {
+            return;
+        }
+        if (snapshotSupport.handleSnapshotMessage(message, getSender())) {
             return;
-        } else if (message instanceof ApplyState){
+        }
+
+        if (message instanceof ApplyState) {
             ApplyState applyState = (ApplyState) message;
 
             long elapsedTime = (System.nanoTime() - applyState.getStartTime());
@@ -228,13 +237,13 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 // and recovery shows data missing
                 context.getReplicatedLog().captureSnapshotIfReady(applyState.getReplicatedLogEntry());
 
-                context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior);
+                context.getSnapshotManager().trimLog(context.getLastApplied());
             }
 
-        } else if (message instanceof ApplyJournalEntries){
+        } else if (message instanceof ApplyJournalEntries) {
             ApplyJournalEntries applyEntries = (ApplyJournalEntries) message;
             if(LOG.isDebugEnabled()) {
-                LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex());
+                LOG.debug("{}: Persisting ApplyJournalEntries with index={}", persistenceId(), applyEntries.getToIndex());
             }
 
             persistence().persist(applyEntries, NoopProcedure.instance());
@@ -248,13 +257,94 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             onGetOnDemandRaftStats();
         } else if(message instanceof InitiateCaptureSnapshot) {
             captureSnapshot();
-        } else if(message instanceof SwitchBehavior){
+        } else if(message instanceof SwitchBehavior) {
             switchBehavior(((SwitchBehavior) message));
-        } else if(!snapshotSupport.handleSnapshotMessage(message, getSender())) {
+        } 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));
         }
     }
 
+    private void initiateLeadershipTransfer(final RaftActorLeadershipTransferCohort.OnComplete onComplete) {
+        LOG.debug("{}: Initiating leader transfer", persistenceId());
+
+        if(leadershipTransferInProgress == null) {
+            leadershipTransferInProgress = new RaftActorLeadershipTransferCohort(this, getSender());
+            leadershipTransferInProgress.addOnComplete(new RaftActorLeadershipTransferCohort.OnComplete() {
+                @Override
+                public void onSuccess(ActorRef raftActorRef, ActorRef replyTo) {
+                    leadershipTransferInProgress = null;
+                }
+
+                @Override
+                public void onFailure(ActorRef raftActorRef, ActorRef replyTo) {
+                    leadershipTransferInProgress = null;
+                }
+            });
+
+            leadershipTransferInProgress.addOnComplete(onComplete);
+            leadershipTransferInProgress.init();
+        } else {
+            LOG.debug("{}: prior leader transfer in progress - adding callback", persistenceId());
+            leadershipTransferInProgress.addOnComplete(onComplete);
+        }
+    }
+
+    private void onShutDown() {
+        LOG.debug("{}: onShutDown", persistenceId());
+
+        if(shuttingDown) {
+            return;
+        }
+
+        shuttingDown = true;
+
+        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);
+                }
+
+                @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());
+                }
+
+                @Override
+                protected void doCancel() {
+                    self().tell(PoisonPill.getInstance(), self());
+                }
+            });
+        } else {
+            self().tell(PoisonPill.getInstance(), self());
+        }
+    }
+
+    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());
+        }
+    }
+
     private void switchBehavior(SwitchBehavior message) {
         if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
             RaftState newState = message.getNewState();
@@ -275,9 +365,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
     }
 
-    protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
-        return new RaftActorSnapshotMessageSupport(context, currentBehavior,
-                getRaftActorSnapshotCohort());
+    @VisibleForTesting
+    RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() {
+        return new RaftActorSnapshotMessageSupport(context, getRaftActorSnapshotCohort());
     }
 
     private void onGetOnDemandRaftStats() {
@@ -288,6 +378,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             peerAddresses.put(peerId, context.getPeerAddress(peerId));
         }
 
+        final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
         OnDemandRaftState.Builder builder = OnDemandRaftState.builder()
                 .commitIndex(context.getCommitIndex())
                 .currentTerm(context.getTermInformation().getCurrentTerm())
@@ -303,9 +394,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 .snapshotIndex(replicatedLog().getSnapshotIndex())
                 .snapshotTerm(replicatedLog().getSnapshotTerm())
                 .votedFor(context.getTermInformation().getVotedFor())
-                .peerAddresses(peerAddresses);
+                .peerAddresses(peerAddresses)
+                .customRaftPolicyClassName(context.getConfigParams().getCustomRaftPolicyImplementationClass());
 
-        ReplicatedLogEntry lastLogEntry = getLastLogEntry();
+        ReplicatedLogEntry lastLogEntry = replicatedLog().last();
         if (lastLogEntry != null) {
             builder.lastLogIndex(lastLogEntry.getIndex());
             builder.lastLogTerm(lastLogEntry.getTerm());
@@ -348,6 +440,12 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             }
 
             onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId());
+
+            if(leadershipTransferInProgress != null) {
+                leadershipTransferInProgress.onNewLeader(currentBehavior.getLeaderId());
+            }
+
+            serverConfigurationSupport.onNewLeader(currentBehavior.getLeaderId());
         }
 
         if (roleChangeNotifier.isPresent() &&
@@ -411,7 +509,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                     context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
 
                     // Send message for replication
-                    currentBehavior.handleMessage(getSelf(),
+                    getCurrentBehavior().handleMessage(getSelf(),
                             new Replicate(clientActor, identifier, replicatedLogEntry));
                 }
             }
@@ -428,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();
     }
 
     /**
@@ -442,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();
     }
 
     /**
@@ -466,30 +573,19 @@ 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;
     }
@@ -502,12 +598,24 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         String newRaftPolicy = configParams.
             getCustomRaftPolicyImplementationClass();
 
-        LOG.debug ("RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}",
+        LOG.debug("{}: RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}", persistenceId(),
             oldRaftPolicy, newRaftPolicy);
         context.setConfigParams(configParams);
         if (!Objects.equal(oldRaftPolicy, newRaftPolicy)) {
-            //RaftPolicy is modifed for the Actor. Re-initialize its current behaviour
-            initializeBehavior();
+            // 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();
+            }
         }
     }
 
@@ -614,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;
         }
@@ -640,11 +765,42 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     private void captureSnapshot() {
         SnapshotManager snapshotManager = context.getSnapshotManager();
 
-        if(!snapshotManager.isCapturing()) {
+        if (!snapshotManager.isCapturing()) {
+            final long idx = getCurrentBehavior().getReplicatedToAllIndex();
             LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}",
-                replicatedLog().last(), currentBehavior.getReplicatedToAllIndex());
+                replicatedLog().last(), idx);
+
+            snapshotManager.capture(replicatedLog().last(), idx);
+        }
+    }
+
+    /**
+     * 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();
+                }
 
-            snapshotManager.capture(replicatedLog().last(), currentBehavior.getReplicatedToAllIndex());
+                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();
+                    }
+                }
+            });
         }
     }
 
@@ -734,9 +890,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         @Override
         public RaftActorBehavior get() {
             if(this.message instanceof SwitchBehavior){
-                return ((SwitchBehavior) message).getNewState().createBehavior(getRaftActorContext());
+                return AbstractRaftActorBehavior.createBehavior(context, ((SwitchBehavior) message).getNewState());
             }
-            return currentBehavior.handleMessage(sender, message);
+            return getCurrentBehavior().handleMessage(sender, message);
         }
     }
 }