X-Git-Url: https://git.opendaylight.org/gerrit/gitweb?a=blobdiff_plain;f=opendaylight%2Fmd-sal%2Fsal-akka-raft%2Fsrc%2Fmain%2Fjava%2Forg%2Fopendaylight%2Fcontroller%2Fcluster%2Fraft%2FRaftActor.java;h=e9e0d630d092eeffa1833441936c26cfeea40d17;hb=c68d251880d95d6d2f8df70c67d2cdd3a3a47685;hp=157a53ed2d1797b662f6fadccd3acf4027ae442f;hpb=f9a9cd1ea40d2477ccb16b03c71a87595226595a;p=controller.git 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 157a53ed2d..e9e0d630d0 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 @@ -16,10 +16,11 @@ 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 +35,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; @@ -114,12 +117,10 @@ 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(); public RaftActor(String id, Map peerAddresses, - Optional configParams) { + Optional configParams, short payloadVersion) { context = new RaftActorContextImpl(this.getSelf(), this.getContext(), id, new ElectionTermImpl(delegatingPersistenceProvider, id, LOG), @@ -127,7 +128,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { (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 @@ -136,6 +138,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { context.getConfigParams().getJournalRecoveryLogBatchSize()); super.preStart(); + + snapshotSupport = newRaftActorSnapshotMessageSupport(); } @Override @@ -177,8 +181,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } protected RaftActorRecoverySupport newRaftActorRecoverySupport() { - return new RaftActorRecoverySupport(delegatingPersistenceProvider, context, currentBehavior, - getRaftActorRecoveryCohort()); + return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort()); } protected void initializeBehavior(){ @@ -192,16 +195,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } @Override - public void handleCommand(Object message) { - if(snapshotSupport == null) { - snapshotSupport = newRaftActorSnapshotMessageSupport(); - } - - boolean handled = snapshotSupport.handleSnapshotMessage(message); - if(handled) { - return; - } - + public void handleCommand(final Object message) { if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; @@ -220,6 +214,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()) { @@ -235,18 +239,38 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { ); } else if(message instanceof GetOnDemandRaftState) { onGetOnDemandRaftStats(); - } else { - 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)) { + 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(delegatingPersistenceProvider, context, - currentBehavior, getRaftActorSnapshotCohort()); + return new RaftActorSnapshotMessageSupport(context, currentBehavior, + getRaftActorSnapshotCohort()); } private void onGetOnDemandRaftStats() { @@ -267,7 +291,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { .snapshotIndex(replicatedLog().getSnapshotIndex()) .snapshotTerm(replicatedLog().getSnapshotTerm()) .votedFor(context.getTermInformation().getVotedFor()) - .peerAddresses(ImmutableMap.copyOf(context.getPeerAddresses())); + .peerAddresses(new HashMap<>(context.getPeerAddresses())); ReplicatedLogEntry lastLogEntry = getLastLogEntry(); if (lastLogEntry != null) { @@ -304,9 +328,11 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { // 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(oldBehaviorLeaderId, 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()); @@ -319,6 +345,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. @@ -343,21 +383,21 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure() { @Override public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception { - if(!hasFollowers()){ + 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)); @@ -572,14 +612,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) { @@ -591,8 +643,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; @@ -613,10 +671,12 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { private static class BehaviorStateHolder { private RaftActorBehavior behavior; private String leaderId; + private short leaderPayloadVersion; void init(RaftActorBehavior behavior) { this.behavior = behavior; this.leaderId = behavior != null ? behavior.getLeaderId() : null; + this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1; } RaftActorBehavior getBehavior() { @@ -626,5 +686,28 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { String getLeaderId() { return leaderId; } + + 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); + } } }