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=5f6f3ec24e573c4dedba6ba42a38dbbe28a51698;hp=4485f3b14457c5f61fe614eaaec226218d0ab3a4;hb=f1c3050779d7770ef6a12a67a1870765c3dfd9eb;hpb=107324809285bfbb9890cba38ffa18390f8de4bd 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 4485f3b144..5f6f3ec24e 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 @@ -12,14 +12,14 @@ package org.opendaylight.controller.cluster.raft; import akka.actor.ActorRef; import akka.actor.ActorSelection; import akka.japi.Procedure; -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,7 +34,9 @@ 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.ApplyState; +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; @@ -106,7 +108,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { */ private final RaftActorContextImpl context; - private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null); + private final DelegatingPersistentDataProvider delegatingPersistenceProvider; + + private final PersistentDataProvider persistentProvider; private RaftActorRecoverySupport raftRecovery; @@ -114,19 +118,23 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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()), delegatingPersistenceProvider, LOG); + context.setPayloadVersion(payloadVersion); context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior)); } @@ -138,6 +146,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { super.preStart(); snapshotSupport = newRaftActorSnapshotMessageSupport(); + serverConfigurationSupport = new RaftActorServerConfigurationSupport(getRaftActorContext()); } @Override @@ -159,22 +168,20 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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"); + } } } @@ -193,8 +200,10 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } @Override - public void handleCommand(Object message) { - if (message instanceof ApplyState){ + 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()); @@ -212,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()) { @@ -227,15 +246,35 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { ); } else if(message instanceof GetOnDemandRaftState) { onGetOnDemandRaftStats(); - } else if(!snapshotSupport.handleSnapshotMessage(message)) { - reusableBehaviorStateHolder.init(getCurrentBehavior()); - - setCurrentBehavior(currentBehavior.handleMessage(getSender(), message)); + } 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)); + } + } - handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + private void switchBehavior(SwitchBehavior message) { + if(!getRaftActorContext().getRaftPolicy().automaticElectionsEnabled()) { + RaftState newState = message.getNewState(); + if( newState == RaftState.Leader || newState == RaftState.Follower) { + switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message)); + getRaftActorContext().getTermInformation().updateAndPersist(message.getNewTerm(), ""); + } else { + LOG.warn("Switching to behavior : {} - not supported", newState); + } } } + private void switchBehavior(Supplier supplier){ + reusableBehaviorStateHolder.init(getCurrentBehavior()); + + setCurrentBehavior(supplier.get()); + + handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + } + protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() { return new RaftActorSnapshotMessageSupport(context, currentBehavior, getRaftActorSnapshotCohort()); @@ -244,6 +283,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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()) @@ -259,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) { @@ -291,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() && @@ -311,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. @@ -334,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)); @@ -435,7 +495,20 @@ 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 {}", + oldRaftPolicy, newRaftPolicy); context.setConfigParams(configParams); + if (!Objects.equal(oldRaftPolicy, newRaftPolicy)) { + //RaftPolicy is modifed for the Actor. Re-initialize its current behaviour + initializeBehavior(); + } } public final DataPersistenceProvider persistence() { @@ -564,14 +637,26 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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.DeleteEntriesTest} + * @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) { @@ -583,8 +668,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; @@ -604,19 +695,48 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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); } } }