Remove snapshot after startup and fix related bug
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / RaftActor.java
index 8d5536f9245b29335c51e6d94e5a87f755ad99d9..602a76ba4215e9fa29b1a187d37547b6a6aa8285 100644 (file)
@@ -12,19 +12,20 @@ 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;
 import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Verify;
 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.Objects;
 import java.util.concurrent.TimeUnit;
-import java.util.function.Supplier;
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.apache.commons.lang3.time.DurationFormatUtils;
 import org.opendaylight.controller.cluster.DataPersistenceProvider;
 import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider;
@@ -50,6 +51,8 @@ import org.opendaylight.controller.cluster.raft.client.messages.GetOnDemandRaftS
 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.opendaylight.yangtools.concepts.Identifier;
+import org.opendaylight.yangtools.concepts.Immutable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -109,14 +112,12 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     private final PersistentDataProvider persistentProvider;
 
+    private final BehaviorStateTracker behaviorStateTracker = new BehaviorStateTracker();
+
     private RaftActorRecoverySupport raftRecovery;
 
     private RaftActorSnapshotMessageSupport snapshotSupport;
 
-    private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder();
-
-    private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier();
-
     private RaftActorServerConfigurationSupport serverConfigurationSupport;
 
     private RaftActorLeadershipTransferCohort leadershipTransferInProgress;
@@ -169,13 +170,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             initializeBehavior();
 
             raftRecovery = null;
-
-            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());
-            }
         }
     }
 
@@ -189,21 +183,39 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
     }
 
     @VisibleForTesting
-    protected void changeCurrentBehavior(RaftActorBehavior newBehavior){
-        if(getCurrentBehavior() != null) {
+    protected void changeCurrentBehavior(RaftActorBehavior newBehavior) {
+        final RaftActorBehavior currentBehavior = getCurrentBehavior();
+        if (currentBehavior != null) {
             try {
-                getCurrentBehavior().close();
-            } catch(Exception e) {
-                LOG.warn("{}: Error closing behavior {}", persistence(), getCurrentBehavior(), e);
+                currentBehavior.close();
+            } catch (Exception e) {
+                LOG.warn("{}: Error closing behavior {}", persistence(), currentBehavior, e);
             }
         }
 
-        reusableBehaviorStateHolder.init(getCurrentBehavior());
+        final BehaviorState state = behaviorStateTracker.capture(currentBehavior);
         setCurrentBehavior(newBehavior);
-        handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
+        handleBehaviorChange(state, newBehavior);
+    }
+
+    /**
+     * Method exposed for subclasses to plug-in their logic. This method is invoked by {@link #handleCommand(Object)}
+     * for messages which are not handled by this class. Subclasses overriding this class should fall back to this
+     * implementation for messages which they do not handle
+     *
+     * @param message Incoming command message
+     */
+    protected void handleNonRaftCommand(final Object message) {
+        unhandled(message);
     }
 
+    /**
+     * @deprecated This method is not final for testing purposes. DO NOT OVERRIDE IT, override
+     * {@link #handleNonRaftCommand(Object)} instead.
+     */
+    @Deprecated
     @Override
+    // FIXME: make this method final once our unit tests do not need to override it
     protected void handleCommand(final Object message) {
         if (serverConfigurationSupport.handleMessage(message, getSender())) {
             return;
@@ -215,11 +227,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         if (message instanceof ApplyState) {
             ApplyState applyState = (ApplyState) message;
 
-            long elapsedTime = (System.nanoTime() - applyState.getStartTime());
-            if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
-                LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
-                        TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
-            }
+            long startTime = System.nanoTime();
 
             if(LOG.isDebugEnabled()) {
                 LOG.debug("{}: Applying state for log index {} data {}",
@@ -230,6 +238,12 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             applyState(applyState.getClientActor(), applyState.getIdentifier(),
                 applyState.getReplicatedLogEntry().getData());
 
+            long elapsedTime = System.nanoTime() - startTime;
+            if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){
+                LOG.debug("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}",
+                        TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState);
+            }
+
             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
@@ -266,7 +280,19 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         } else if(message instanceof Runnable) {
             ((Runnable)message).run();
         } else {
-            switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
+            // Processing the message may affect the state, hence we need to capture it
+            final RaftActorBehavior currentBehavior = getCurrentBehavior();
+            final BehaviorState state = behaviorStateTracker.capture(currentBehavior);
+
+            // A behavior indicates that it processed the change by returning a reference to the next behavior
+            // to be used. A null return indicates it has not processed the message and we should be passing it to
+            // the subclass for handling.
+            final RaftActorBehavior nextBehavior = currentBehavior.handleMessage(getSender(), message);
+            if (nextBehavior != null) {
+                switchBehavior(state, nextBehavior);
+            } else {
+                handleNonRaftCommand(message);
+            }
         }
     }
 
@@ -353,7 +379,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) {
             RaftState newState = message.getNewState();
             if( newState == RaftState.Leader || newState == RaftState.Follower) {
-                switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message));
+                switchBehavior(behaviorStateTracker.capture(getCurrentBehavior()),
+                    AbstractRaftActorBehavior.createBehavior(context, message.getNewState()));
                 getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), "");
             } else {
                 LOG.warn("Switching to behavior : {} - not supported", newState);
@@ -361,12 +388,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-    private void switchBehavior(Supplier<RaftActorBehavior> supplier){
-        reusableBehaviorStateHolder.init(getCurrentBehavior());
-
-        setCurrentBehavior(supplier.get());
-
-        handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior());
+    private void switchBehavior(final BehaviorState oldBehaviorState, final RaftActorBehavior nextBehavior) {
+        setCurrentBehavior(nextBehavior);
+        handleBehaviorChange(oldBehaviorState, nextBehavior);
     }
 
     @VisibleForTesting
@@ -378,8 +402,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         // Debugging message to retrieve raft stats.
 
         Map<String, String> peerAddresses = new HashMap<>();
-        for(String peerId: context.getPeerIds()) {
-            peerAddresses.put(peerId, context.getPeerAddress(peerId));
+        Map<String, Boolean> peerVotingStates = new HashMap<>();
+        for(PeerInfo info: context.getPeers()) {
+            peerVotingStates.put(info.getId(), info.isVoting());
+            peerAddresses.put(info.getId(), info.getAddress() != null ? info.getAddress() : "");
         }
 
         final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
@@ -398,7 +424,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
                 .snapshotIndex(replicatedLog().getSnapshotIndex())
                 .snapshotTerm(replicatedLog().getSnapshotTerm())
                 .votedFor(context.getTermInformation().getVotedFor())
+                .isVoting(context.isVotingMember())
                 .peerAddresses(peerAddresses)
+                .peerVotingStates(peerVotingStates)
                 .customRaftPolicyClassName(context.getConfigParams().getCustomRaftPolicyImplementationClass());
 
         ReplicatedLogEntry lastLogEntry = replicatedLog().last();
@@ -414,7 +442,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
             for(String id: followerIds) {
                 final FollowerLogInformation info = leader.getFollower(id);
                 followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(),
-                        info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity())));
+                        info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity()),
+                        context.getPeerInfo(info.getId()).isVoting()));
             }
 
             builder.followerInfoList(followerInfoList);
@@ -424,7 +453,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
     }
 
-    private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) {
+    private void handleBehaviorChange(BehaviorState oldBehaviorState, RaftActorBehavior currentBehavior) {
         RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior();
 
         if (oldBehavior != currentBehavior){
@@ -436,7 +465,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
         // it can happen that the state has not changed but the leader has changed.
         Optional<ActorRef> roleChangeNotifier = getRoleChangeNotifier();
-        if(!Objects.equal(lastValidLeaderId, currentBehavior.getLeaderId()) ||
+        if(!Objects.equals(lastValidLeaderId, currentBehavior.getLeaderId()) ||
            oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) {
             if(roleChangeNotifier.isPresent()) {
                 roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(),
@@ -481,8 +510,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      * @param identifier
      * @param data
      */
-    protected void persistData(final ActorRef clientActor, final String identifier,
-        final Payload data) {
+    protected final void persistData(final ActorRef clientActor, final Identifier identifier, final Payload data) {
 
         ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry(
             context.getReplicatedLog().lastIndex() + 1,
@@ -494,28 +522,25 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
 
         final RaftActorContext raftContext = getRaftActorContext();
 
-        replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure<ReplicatedLogEntry>() {
-            @Override
-            public void apply(ReplicatedLogEntry replicatedLogEntry) {
-                if (!hasFollowers()){
-                    // Increment the Commit Index and the Last Applied values
-                    raftContext.setCommitIndex(replicatedLogEntry.getIndex());
-                    raftContext.setLastApplied(replicatedLogEntry.getIndex());
+        replicatedLog().appendAndPersist(replicatedLogEntry, replicatedLogEntry1 -> {
+            if (!hasFollowers()){
+                // Increment the Commit Index and the Last Applied values
+                raftContext.setCommitIndex(replicatedLogEntry1.getIndex());
+                raftContext.setLastApplied(replicatedLogEntry1.getIndex());
 
-                    // Apply the state immediately.
-                    self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry), self());
+                // Apply the state immediately.
+                self().tell(new ApplyState(clientActor, identifier, replicatedLogEntry1), 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());
+                // Send a ApplyJournalEntries message so that we write the fact that we applied
+                // the state to durable storage
+                self().tell(new ApplyJournalEntries(replicatedLogEntry1.getIndex()), self());
 
-                } else if (clientActor != null) {
-                    context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry);
+            } else if (clientActor != null) {
+                context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry1);
 
-                    // Send message for replication
-                    getCurrentBehavior().handleMessage(getSelf(),
-                            new Replicate(clientActor, identifier, replicatedLogEntry));
-                }
+                // Send message for replication
+                getCurrentBehavior().handleMessage(getSelf(),
+                        new Replicate(clientActor, identifier, replicatedLogEntry1));
             }
         });
     }
@@ -605,13 +630,13 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         LOG.debug("{}: RaftPolicy used with prev.config {}, RaftPolicy used with newConfig {}", persistenceId(),
             oldRaftPolicy, newRaftPolicy);
         context.setConfigParams(configParams);
-        if (!Objects.equal(oldRaftPolicy, newRaftPolicy)) {
+        if (!Objects.equals(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();
+            if (behavior != null && behavior.state() == RaftState.Follower) {
+                String previousLeaderId = behavior.getLeaderId();
                 short previousLeaderPayloadVersion = behavior.getLeaderPayloadVersion();
 
                 LOG.debug("{}: Re-initializing to Follower with previous leaderId {}", persistenceId(), previousLeaderId);
@@ -693,8 +718,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
      * @param data        A piece of data that was persisted by the persistData call.
      *                    This should NEVER be null.
      */
-    protected abstract void applyState(ActorRef clientActor, String identifier,
-        Object data);
+    protected abstract void applyState(ActorRef clientActor, Identifier identifier, Object data);
 
     /**
      * Returns the RaftActorRecoveryCohort to participate in persistence recovery.
@@ -853,50 +877,88 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor {
         }
     }
 
-    private static class BehaviorStateHolder {
-        private RaftActorBehavior behavior;
-        private String lastValidLeaderId;
-        private short leaderPayloadVersion;
+    /**
+     * A point-in-time capture of {@link RaftActorBehavior} state critical for transitioning between behaviors.
+     */
+    private static abstract class BehaviorState implements Immutable {
+        @Nullable abstract RaftActorBehavior getBehavior();
+        @Nullable abstract String getLastValidLeaderId();
+        @Nullable abstract short getLeaderPayloadVersion();
+    }
 
-        void init(RaftActorBehavior behavior) {
-            this.behavior = behavior;
-            this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1;
+    /**
+     * A {@link BehaviorState} corresponding to non-null {@link RaftActorBehavior} state.
+     */
+    private static final class SimpleBehaviorState extends BehaviorState {
+        private final RaftActorBehavior behavior;
+        private final String lastValidLeaderId;
+        private final short leaderPayloadVersion;
 
-            String behaviorLeaderId = behavior != null ? behavior.getLeaderId() : null;
-            if(behaviorLeaderId != null) {
-                this.lastValidLeaderId = behaviorLeaderId;
-            }
+        SimpleBehaviorState(final String lastValidLeaderId, final RaftActorBehavior behavior) {
+            this.lastValidLeaderId = lastValidLeaderId;
+            this.behavior = Preconditions.checkNotNull(behavior);
+            this.leaderPayloadVersion = behavior.getLeaderPayloadVersion();
         }
 
+        @Override
         RaftActorBehavior getBehavior() {
             return behavior;
         }
 
+        @Override
         String getLastValidLeaderId() {
             return lastValidLeaderId;
         }
 
+        @Override
         short getLeaderPayloadVersion() {
             return leaderPayloadVersion;
         }
     }
 
-    private class SwitchBehaviorSupplier implements Supplier<RaftActorBehavior> {
-        private Object message;
-        private ActorRef sender;
+    /**
+     * Class tracking behavior-related information, which we need to keep around and pass across behavior switches.
+     * An instance is created for each RaftActor. It has two functions:
+     * - it keeps track of the last leader ID we have encountered since we have been created
+     * - it creates state capture needed to transition from one behavior to the next
+     */
+    private static final class BehaviorStateTracker {
+        /**
+         * A {@link BehaviorState} corresponding to null {@link RaftActorBehavior} state. Since null behavior is only
+         * allowed before we receive the first message, we know the leader ID to be null.
+         */
+        private static final BehaviorState NULL_BEHAVIOR_STATE = new BehaviorState() {
+            @Override
+            RaftActorBehavior getBehavior() {
+                return null;
+            }
 
-        public SwitchBehaviorSupplier handleMessage(ActorRef sender, Object message){
-            this.sender = sender;
-            this.message = message;
-            return this;
-        }
+            @Override
+            String getLastValidLeaderId() {
+                return null;
+            }
 
-        @Override
-        public RaftActorBehavior get() {
-            if(this.message instanceof SwitchBehavior){
-                return AbstractRaftActorBehavior.createBehavior(context, ((SwitchBehavior) message).getNewState());
+            @Override
+            short getLeaderPayloadVersion() {
+                return -1;
             }
-            return getCurrentBehavior().handleMessage(sender, message);
+        };
+
+        private String lastValidLeaderId;
+
+        BehaviorState capture(final RaftActorBehavior behavior) {
+            if (behavior == null) {
+                Verify.verify(lastValidLeaderId == null, "Null behavior with non-null last leader");
+                return NULL_BEHAVIOR_STATE;
+            }
+
+            final String leaderId = behavior.getLeaderId();
+            if (leaderId != null) {
+                lastValidLeaderId = leaderId;
+            }
+
+            return new SimpleBehaviorState(lastValidLeaderId, behavior);
         }
     }
+
 }