X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?p=controller.git;a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActor.java;h=47c8db6006544b13c11b42f7abdb370590fd3b8b;hp=1c30fe23175b5af62ffe808d9428c3361688a4b0;hb=f276ae33b951d173b51c467bb7bb1a5f5cf9a1e6;hpb=59e84273be97b1115fb3fb0f06364b7491a40385 diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java index 1c30fe2317..47c8db6006 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/RaftActor.java @@ -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,22 @@ 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.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 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; @@ -36,14 +36,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,7 +50,9 @@ 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.opendaylight.yangtools.concepts.Immutable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -99,51 +100,44 @@ 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 createSnapshotProcedure = new CreateSnapshotProcedure(); + private final PersistentDataProvider persistentProvider; - private Stopwatch recoveryTimer; + private final BehaviorStateTracker behaviorStateTracker = new BehaviorStateTracker(); - private int currentRecoveryBatchCount; + private RaftActorRecoverySupport raftRecovery; - private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder(); + private RaftActorSnapshotMessageSupport snapshotSupport; - public RaftActor(String id, Map peerAddresses) { - this(id, peerAddresses, Optional.absent()); - } + private RaftActorServerConfigurationSupport serverConfigurationSupport; + + private RaftActorLeadershipTransferCohort leadershipTransferInProgress; + + private boolean shuttingDown; public RaftActor(String id, Map peerAddresses, - Optional configParams) { + Optional 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,172 +146,95 @@ 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(); - } - } - } - - private void onRecoveredSnapshot(SnapshotOffer offer) { - if(LOG.isDebugEnabled()) { - LOG.debug("{}: SnapshotOffer called..", persistenceId()); + protected void handleRecover(Object message) { + if(raftRecovery == null) { + raftRecovery = newRaftActorRecoverySupport(); } - initRecoveryTimer(); + boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider); + if(recoveryComplete) { + onRecoveryComplete(); - Snapshot snapshot = (Snapshot) offer.snapshot(); + initializeBehavior(); - // 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 + raftRecovery = null; - 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()); + 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()); + } } - - 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); + protected RaftActorRecoverySupport newRaftActorRecoverySupport() { + return new RaftActorRecoverySupport(context, getRaftActorRecoveryCohort()); } - private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) { - initRecoveryTimer(); + @VisibleForTesting + void initializeBehavior(){ + changeCurrentBehavior(new Follower(context)); + } - int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize(); - if(currentRecoveryBatchCount == 0) { - startLogRecoveryBatch(batchSize); + @VisibleForTesting + protected void changeCurrentBehavior(RaftActorBehavior newBehavior) { + final RaftActorBehavior currentBehavior = getCurrentBehavior(); + if (currentBehavior != null) { + try { + currentBehavior.close(); + } catch (Exception e) { + LOG.warn("{}: Error closing behavior {}", persistence(), currentBehavior, e); + } } - appendRecoveredLogEntry(logEntry.getData()); - - if(++currentRecoveryBatchCount >= batchSize) { - endCurrentLogRecoveryBatch(); - } + final BehaviorState state = behaviorStateTracker.capture(currentBehavior); + setCurrentBehavior(newBehavior); + handleBehaviorChange(state, newBehavior); } - private void endCurrentLogRecoveryBatch() { - applyCurrentLogRecoveryBatch(); - currentRecoveryBatchCount = 0; + /** + * 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); } - private void onRecoveryCompletedMessage() { - if(currentRecoveryBatchCount > 0) { - endCurrentLogRecoveryBatch(); + /** + * @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; } - - onRecoveryComplete(); - - String recoveryTime = ""; - if(recoveryTimer != null) { - recoveryTimer.stop(); - recoveryTime = " in " + recoveryTimer.toString(); - recoveryTimer = null; + if (snapshotSupport.handleSnapshotMessage(message, getSender())) { + return; } - 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 void initializeBehavior(){ - changeCurrentBehavior(new Follower(context)); - } - - protected void changeCurrentBehavior(RaftActorBehavior newBehavior){ - reusableBehaviorStateHolder.init(getCurrentBehavior()); - setCurrentBehavior(newBehavior); - handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); - } - - @Override public void handleCommand(Object message) { - if (message instanceof ApplyState){ + 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 {}", @@ -328,76 +245,175 @@ 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()); + 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); } - persistence().persist(applyEntries, NoopProcedure.instance()); + 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()); - } 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 { + // 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); + } + } + } - } 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); + leadershipTransferInProgress.addOnComplete(new RaftActorLeadershipTransferCohort.OnComplete() { + @Override + public void onSuccess(ActorRef raftActorRef) { + leadershipTransferInProgress = null; + } - commitSnapshot(sequenceNumber); + @Override + public void onFailure(ActorRef raftActorRef) { + 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; + } - } else if (message instanceof CaptureSnapshot) { - LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message); + shuttingDown = true; - context.getSnapshotManager().create(createSnapshotProcedure); + final RaftActorBehavior currentBehavior = context.getCurrentBehavior(); + if (currentBehavior.state() != RaftState.Leader) { + // For non-leaders shutdown is a no-op + self().tell(PoisonPill.getInstance(), self()); + return; + } - } else if (message instanceof CaptureSnapshotReply) { - handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot()); - } else if(message instanceof GetOnDemandRaftState) { - onGetOnDemandRaftStats(); - } else if (message.equals(COMMIT_SNAPSHOT)) { - commitSnapshot(-1); + if (context.hasFollowers()) { + initiateLeadershipTransfer(new RaftActorLeadershipTransferCohort.OnComplete() { + @Override + public void onSuccess(ActorRef raftActorRef) { + LOG.debug("{}: leader transfer succeeded - sending PoisonPill", persistenceId()); + raftActorRef.tell(PoisonPill.getInstance(), raftActorRef); + } + + @Override + public void onFailure(ActorRef raftActorRef) { + LOG.debug("{}: leader transfer failed - sending PoisonPill", persistenceId()); + raftActorRef.tell(PoisonPill.getInstance(), raftActorRef); + } + }); } else { - reusableBehaviorStateHolder.init(getCurrentBehavior()); + pauseLeader(new TimedRunnable(context.getConfigParams().getElectionTimeOutInterval(), this) { + @Override + protected void doRun() { + self().tell(PoisonPill.getInstance(), self()); + } - setCurrentBehavior(currentBehavior.handleMessage(getSender(), message)); + @Override + protected void doCancel() { + self().tell(PoisonPill.getInstance(), self()); + } + }); + } + } - handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + private void onLeaderTransitioning() { + LOG.debug("{}: onLeaderTransitioning", persistenceId()); + Optional 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(); + if( newState == RaftState.Leader || newState == RaftState.Follower) { + switchBehavior(behaviorStateTracker.capture(getCurrentBehavior()), + AbstractRaftActorBehavior.createBehavior(context, message.getNewState())); + getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), ""); + } else { + LOG.warn("Switching to behavior : {} - not supported", newState); + } + } + } + + private void switchBehavior(final BehaviorState oldBehaviorState, final RaftActorBehavior nextBehavior) { + setCurrentBehavior(nextBehavior); + handleBehaviorChange(oldBehaviorState, nextBehavior); + } + + @VisibleForTesting + RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() { + return new RaftActorSnapshotMessageSupport(context, getRaftActorSnapshotCohort()); + } + private void onGetOnDemandRaftStats() { // Debugging message to retrieve raft stats. + Map 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 +429,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()); @@ -438,24 +455,32 @@ 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){ 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 roleChangeNotifier = getRoleChangeNotifier(); - if(!Objects.equal(oldBehaviorLeaderId, currentBehavior.getLeaderId())) { + if(!Objects.equals(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 +490,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 +527,24 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure() { @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 +561,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 +575,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 +608,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.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 != null && behavior.state() == RaftState.Follower) { + String previousLeaderId = 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 +683,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 +707,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 +728,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. + * Returns the RaftActorRecoveryCohort to participate in persistence recovery. */ - 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}. - */ - protected abstract void applyCurrentLogRecoveryBatch(); + @Nonnull + protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort(); /** * This method is called when recovery is complete. @@ -702,24 +739,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. - *

- * 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 - */ - 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 + * Returns the RaftActorSnapshotCohort to participate in persistence recovery. */ - 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 +757,30 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { */ protected abstract Optional 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. Important: the run method must be called on + * this actor's thread dispatcher as as it modifies internal state. + *

+ * 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 +793,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(); - context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory()); + if (!snapshotManager.isCapturing()) { + final long idx = getCurrentBehavior().getReplicatedToAllIndex(); + LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}", + replicatedLog().last(), idx); + + 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) { + LOG.debug("{}: leader transfer succeeded after change to non-voting", persistenceId()); + ensureFollowerState(); + } + + @Override + public void onFailure(ActorRef raftActorRef) { + 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 +859,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 +884,88 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } } - private class CreateSnapshotProcedure implements Procedure { - - @Override - public void apply(Void aVoid) throws Exception { - createSnapshot(); - } + /** + * 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(); } - private static class BehaviorStateHolder { - private RaftActorBehavior behavior; - private String leaderId; + /** + * 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; - void init(RaftActorBehavior behavior) { - this.behavior = behavior; - this.leaderId = behavior != null ? behavior.getLeaderId() : null; + SimpleBehaviorState(final String lastValidLeaderId, final RaftActorBehavior behavior) { + this.lastValidLeaderId = lastValidLeaderId; + this.behavior = Preconditions.checkNotNull(behavior); + this.leaderPayloadVersion = behavior.getLeaderPayloadVersion(); } + @Override RaftActorBehavior getBehavior() { return behavior; } - String getLeaderId() { - return leaderId; + @Override + String getLastValidLeaderId() { + return lastValidLeaderId; + } + + @Override + short getLeaderPayloadVersion() { + return leaderPayloadVersion; + } + } + + /** + * 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; + } + + @Override + String getLastValidLeaderId() { + return null; + } + + @Override + short getLeaderPayloadVersion() { + return -1; + } + }; + + 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); } } + }