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=b93ea4dd8a8b3e6028d2943f87c84d17ee3eb01a;hp=1f1521d797d45790dc35c747ca9f8af2d4c6dd8d;hb=8882e6077db69d22bcc57fcf12dd4a02a81a4967;hpb=5de57714fa057ac80f930c2fcce2758ca0a5f514 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 1f1521d797..b93ea4dd8a 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, @@ -11,16 +12,14 @@ package org.opendaylight.controller.cluster.raft; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.japi.Procedure; -import akka.persistence.SaveSnapshotFailure; -import akka.persistence.SaveSnapshotSuccess; -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.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; @@ -34,11 +33,10 @@ 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.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.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.Follower; @@ -96,8 +94,6 @@ 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()); /** @@ -112,27 +108,34 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { */ 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 RaftActorRecoverySupport raftRecovery; + private RaftActorSnapshotMessageSupport snapshotSupport; + private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder(); - public RaftActor(String id, Map peerAddresses) { - this(id, peerAddresses, Optional.absent()); - } + private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier(); + + private RaftActorServerConfigurationSupport serverConfigurationSupport; 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)); + context.setPayloadVersion(payloadVersion); + context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior)); } @Override @@ -141,11 +144,14 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { context.getConfigParams().getJournalRecoveryLogBatchSize()); super.preStart(); + + snapshotSupport = newRaftActorSnapshotMessageSupport(); + serverConfigurationSupport = new RaftActorServerConfigurationSupport(getRaftActorContext()); } @Override public void postStop() { - if(currentBehavior != null) { + if(currentBehavior.getDelegate() != null) { try { currentBehavior.close(); } catch (Exception e) { @@ -159,29 +165,30 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { @Override public void handleRecover(Object message) { if(raftRecovery == null) { - raftRecovery = new RaftActorRecoverySupport(delegatingPersistenceProvider, context, currentBehavior, - getRaftActorRecoveryCohort()); + raftRecovery = newRaftActorRecoverySupport(); } - boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message); + boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider); if(recoveryComplete) { - if(!persistence().isRecoveryApplicable()) { - // 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(); raftRecovery = null; + + if (context.getReplicatedLog().size() > 0) { + self().tell(new InitiateCaptureSnapshot(), self()); + LOG.info("Snapshot capture initiated after recovery"); + } else { + LOG.info("Snapshot capture NOT initiated after recovery, journal empty"); + } } } + protected RaftActorRecoverySupport newRaftActorRecoverySupport() { + return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort()); + } + protected void initializeBehavior(){ changeCurrentBehavior(new Follower(context)); } @@ -192,8 +199,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); } - @Override public void handleCommand(Object message) { - if (message instanceof ApplyState){ + @Override + public void handleCommand(final Object message) { + if(serverConfigurationSupport.handleMessage(message, this, getSender())) { + return; + } else if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; long elapsedTime = (System.nanoTime() - applyState.getStartTime()); @@ -211,6 +221,16 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { applyState(applyState.getClientActor(), applyState.getIdentifier(), applyState.getReplicatedLogEntry().getData()); + 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()); + + context.getSnapshotManager().trimLog(context.getLastApplied(), currentBehavior); + } + } else if (message instanceof ApplyJournalEntries){ ApplyJournalEntries applyEntries = (ApplyJournalEntries) message; if(LOG.isDebugEnabled()) { @@ -219,68 +239,55 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { persistence().persist(applyEntries, NoopProcedure.instance()); - } else if(message instanceof ApplySnapshot ) { - Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); - - if(LOG.isDebugEnabled()) { - LOG.debug("{}: ApplySnapshot called on Follower Actor " + - "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(), - snapshot.getLastAppliedTerm() - ); - } - - 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()); - } 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(!snapshotSupport.handleSnapshotMessage(message, getSender())) { + switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message)); + } + } - } else if (message instanceof SaveSnapshotSuccess) { - SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; - LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId()); - - long sequenceNumber = success.metadata().sequenceNr(); - - commitSnapshot(sequenceNumber); - - } else if (message instanceof SaveSnapshotFailure) { - SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; - - LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:", - persistenceId(), saveSnapshotFailure.cause()); - - context.getSnapshotManager().rollback(); - - } else if (message instanceof CaptureSnapshot) { - LOG.debug("{}: CaptureSnapshot received by actor: {}", persistenceId(), message); + 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); + } + } + } - context.getSnapshotManager().create(createSnapshotProcedure); + private void switchBehavior(Supplier supplier){ + reusableBehaviorStateHolder.init(getCurrentBehavior()); - } else if (message instanceof CaptureSnapshotReply) { - handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot()); - } else if(message instanceof GetOnDemandRaftState) { - onGetOnDemandRaftStats(); - } else if (message.equals(COMMIT_SNAPSHOT)) { - commitSnapshot(-1); - } else { - reusableBehaviorStateHolder.init(getCurrentBehavior()); + setCurrentBehavior(supplier.get()); - setCurrentBehavior(currentBehavior.handleMessage(getSender(), message)); + handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + } - handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); - } + protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() { + return new RaftActorSnapshotMessageSupport(context, currentBehavior, + 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)); + } + OnDemandRaftState.Builder builder = OnDemandRaftState.builder() .commitIndex(context.getCommitIndex()) .currentTerm(context.getTermInformation().getCurrentTerm()) @@ -296,7 +303,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { .snapshotIndex(replicatedLog().getSnapshotIndex()) .snapshotTerm(replicatedLog().getSnapshotTerm()) .votedFor(context.getTermInformation().getVotedFor()) - .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses())); + .peerAddresses(peerAddresses); ReplicatedLogEntry lastLogEntry = getLastLogEntry(); if (lastLogEntry != null) { @@ -328,17 +335,19 @@ 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 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(oldBehaviorLeaderId, currentBehavior.getLeaderId()); + onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId()); } if (roleChangeNotifier.isPresent() && @@ -348,6 +357,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. @@ -371,22 +394,22 @@ 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(), new Replicate(clientActor, identifier, replicatedLogEntry)); @@ -472,7 +495,31 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } 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 = currentBehavior.getDelegate(); + if(behavior instanceof Follower) { + String previousLeaderId = ((Follower)behavior).getLeaderId(); + + LOG.debug("{}: Re-initializing to Follower with previous leaderId {}", persistenceId(), previousLeaderId); + + changeCurrentBehavior(new Follower(context, previousLeaderId)); + } else { + initializeBehavior(); + } + } } public final DataPersistenceProvider persistence() { @@ -504,7 +551,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()); } }); } @@ -528,10 +575,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 @@ -564,24 +607,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 + * 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 @@ -615,18 +644,30 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { return peerAddress; } - private void handleCaptureSnapshotReply(byte[] snapshotBytes) { - LOG.debug("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length); - - context.getSnapshotManager().persist(persistence(), snapshotBytes, currentBehavior, context.getTotalMemory()); - } - protected boolean hasFollowers(){ return getRaftActorContext().hasFollowers(); } + private void captureSnapshot() { + SnapshotManager snapshotManager = context.getSnapshotManager(); + + if(!snapshotManager.isCapturing()) { + LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}", + replicatedLog().last(), currentBehavior.getReplicatedToAllIndex()); + + snapshotManager.capture(replicatedLog().last(), currentBehavior.getReplicatedToAllIndex()); + } + } + + /** + * @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) { @@ -638,8 +679,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; @@ -657,29 +704,50 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } } - private class CreateSnapshotProcedure implements Procedure { - - @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 { + 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 ((SwitchBehavior) message).getNewState().createBehavior(getRaftActorContext()); + } + return currentBehavior.handleMessage(sender, message); } } }