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=aa7b4533b7758f7a3f457ea48b1ae72cc6d94162;hb=023402c7c80372260b6c5c82f120093a73806717;hp=6e1a13cf0c19669443b9273e1d2703a3ff2dede9;hpb=6e2f13caf52fe4f4af8aa7c0e8ffd475d4e15fba;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 6e1a13cf0c..a8c32cd469 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,38 +11,44 @@ package org.opendaylight.controller.cluster.raft; import akka.actor.ActorRef; import akka.actor.ActorSelection; -import akka.event.Logging; -import akka.event.LoggingAdapter; 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 akka.persistence.UntypedPersistentActor; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; import com.google.common.base.Optional; -import com.google.protobuf.ByteString; -import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries; -import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; +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; +import javax.annotation.Nonnull; +import org.apache.commons.lang3.time.DurationFormatUtils; +import org.opendaylight.controller.cluster.DataPersistenceProvider; +import org.opendaylight.controller.cluster.DelegatingPersistentDataProvider; +import org.opendaylight.controller.cluster.NonPersistentDataProvider; +import org.opendaylight.controller.cluster.PersistentDataProvider; +import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor; +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.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.SendHeartBeat; -import org.opendaylight.controller.cluster.raft.behaviors.Candidate; +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; -import org.opendaylight.controller.cluster.raft.behaviors.Leader; import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; -import org.opendaylight.controller.cluster.raft.client.messages.AddRaftPeer; import org.opendaylight.controller.cluster.raft.client.messages.FindLeader; import org.opendaylight.controller.cluster.raft.client.messages.FindLeaderReply; -import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer; -import org.opendaylight.controller.cluster.raft.messages.AppendEntriesReply; +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.protobuff.client.messages.Payload; -import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages; - -import java.io.Serializable; -import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * RaftActor encapsulates a state machine that needs to be kept synchronized @@ -83,232 +90,268 @@ import java.util.Map; *
  • when a snapshot should be saved
  • * */ -public abstract class RaftActor extends UntypedPersistentActor { - protected final LoggingAdapter LOG = - Logging.getLogger(getContext().system(), this); +public abstract class RaftActor extends AbstractUntypedPersistentActor { + + private static final long APPLY_STATE_DELAY_THRESHOLD_IN_NANOS = TimeUnit.MILLISECONDS.toNanos(50L); // 50 millis + + 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 RaftActorBehavior currentBehavior; + 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 */ - protected RaftActorContext context; + private final RaftActorContextImpl context; - /** - * The in-memory journal - */ - private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl(); + private final DelegatingPersistentDataProvider delegatingPersistenceProvider = new DelegatingPersistentDataProvider(null); - private CaptureSnapshot captureSnapshot = null; + private final PersistentDataProvider persistentProvider; - private volatile boolean hasSnapshotCaptureInitiated = false; + private RaftActorRecoverySupport raftRecovery; - public RaftActor(String id, Map peerAddresses) { - this(id, peerAddresses, Optional.absent()); - } + private RaftActorSnapshotMessageSupport snapshotSupport; + + private final BehaviorStateHolder reusableBehaviorStateHolder = new BehaviorStateHolder(); + + private final SwitchBehaviorSupplier reusableSwitchBehaviorSupplier = new SwitchBehaviorSupplier(); public RaftActor(String id, Map peerAddresses, - Optional configParams) { + Optional configParams, short payloadVersion) { + persistentProvider = new PersistentDataProvider(this); context = new RaftActorContextImpl(this.getSelf(), - this.getContext(), id, new ElectionTermImpl(), - -1, -1, replicatedLog, peerAddresses, + this.getContext(), id, new ElectionTermImpl(persistentProvider, id, LOG), + -1, -1, peerAddresses, (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), - LOG); + delegatingPersistenceProvider, LOG); + + context.setPayloadVersion(payloadVersion); + context.setReplicatedLog(ReplicatedLogImpl.newInstance(context, currentBehavior)); } - @Override public void onReceiveRecover(Object message) { - if (message instanceof SnapshotOffer) { - LOG.info("SnapshotOffer called.."); - SnapshotOffer offer = (SnapshotOffer) message; - Snapshot snapshot = (Snapshot) offer.snapshot(); + @Override + public void preStart() throws Exception { + LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(), + context.getConfigParams().getJournalRecoveryLogBatchSize()); - // 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 - replicatedLog = new ReplicatedLogImpl(snapshot); + super.preStart(); - context.setReplicatedLog(replicatedLog); - context.setLastApplied(snapshot.getLastAppliedIndex()); - context.setCommitIndex(snapshot.getLastAppliedIndex()); + snapshotSupport = newRaftActorSnapshotMessageSupport(); + } - LOG.info("Applied snapshot to replicatedLog. " + - "snapshotIndex={}, snapshotTerm={}, journal-size={}", - replicatedLog.snapshotIndex, replicatedLog.snapshotTerm, - replicatedLog.size() - ); + @Override + public void postStop() { + if(currentBehavior.getDelegate() != null) { + try { + currentBehavior.close(); + } catch (Exception e) { + LOG.debug("{}: Error closing behavior {}", persistenceId(), currentBehavior.state()); + } + } - // Apply the snapshot to the actors state - applySnapshot(ByteString.copyFrom(snapshot.getState())); + super.postStop(); + } - } else if (message instanceof ReplicatedLogEntry) { - ReplicatedLogEntry logEntry = (ReplicatedLogEntry) message; - LOG.info("Received ReplicatedLogEntry for recovery:{}", logEntry.getIndex()); - replicatedLog.append(logEntry); + @Override + public void handleRecover(Object message) { + if(raftRecovery == null) { + raftRecovery = newRaftActorRecoverySupport(); + } - } else if (message instanceof ApplyLogEntries) { - ApplyLogEntries ale = (ApplyLogEntries) message; + boolean recoveryComplete = raftRecovery.handleRecoveryMessage(message, persistentProvider); + if(recoveryComplete) { + onRecoveryComplete(); - LOG.info("Received ApplyLogEntries for recovery, applying to state:{} to {}", - context.getLastApplied() + 1, ale.getToIndex()); + initializeBehavior(); - for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) { - applyState(null, "recovery", replicatedLog.get(i).getData()); - } - context.setLastApplied(ale.getToIndex()); - context.setCommitIndex(ale.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) { - LOG.info( - "RecoveryCompleted - Switching actor to Follower - " + - "Persistence Id = " + persistenceId() + - " Last index in log:{}, snapshotIndex={}, snapshotTerm={}, " + - "journal-size={}", - replicatedLog.lastIndex(), replicatedLog.snapshotIndex, - replicatedLog.snapshotTerm, replicatedLog.size()); - currentBehavior = switchBehavior(RaftState.Follower); - onStateChanged(); + raftRecovery = null; } } - @Override public void onReceiveCommand(Object message) { + protected RaftActorRecoverySupport newRaftActorRecoverySupport() { + return new RaftActorRecoverySupport(context, currentBehavior, getRaftActorRecoveryCohort()); + } + + protected void initializeBehavior(){ + changeCurrentBehavior(new Follower(context)); + } + + protected void changeCurrentBehavior(RaftActorBehavior newBehavior){ + reusableBehaviorStateHolder.init(getCurrentBehavior()); + setCurrentBehavior(newBehavior); + handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + } + + @Override + public void handleCommand(final Object message) { 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); + } + if(LOG.isDebugEnabled()) { - LOG.debug("Applying state for log index {} data {}", - applyState.getReplicatedLogEntry().getIndex(), + LOG.debug("{}: Applying state for log index {} data {}", + persistenceId(), applyState.getReplicatedLogEntry().getIndex(), applyState.getReplicatedLogEntry().getData()); } applyState(applyState.getClientActor(), applyState.getIdentifier(), applyState.getReplicatedLogEntry().getData()); - } else if (message instanceof ApplyLogEntries){ - ApplyLogEntries ale = (ApplyLogEntries) message; - LOG.info("Persisting ApplyLogEntries with index={}", ale.getToIndex()); - persist(new ApplyLogEntries(ale.getToIndex()), new Procedure() { - @Override - public void apply(ApplyLogEntries param) throws Exception { - } - }); + 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(), currentBehavior); + } + } else if (message instanceof ApplyJournalEntries){ + ApplyJournalEntries applyEntries = (ApplyJournalEntries) message; if(LOG.isDebugEnabled()) { - LOG.debug("ApplySnapshot called on Follower Actor " + - "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(), - snapshot.getLastAppliedTerm() - ); + LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), applyEntries.getToIndex()); } - applySnapshot(ByteString.copyFrom(snapshot.getState())); - //clears the followers log, sets the snapshot index to ensure adjusted-index works - replicatedLog = new ReplicatedLogImpl(snapshot); - context.setReplicatedLog(replicatedLog); - 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(!snapshotSupport.handleSnapshotMessage(message)) { + switchBehavior(reusableSwitchBehaviorSupplier.handleMessage(getSender(), message)); + } + } - } else if (message instanceof SaveSnapshotSuccess) { - SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; - LOG.info("SaveSnapshotSuccess received for snapshot"); - - context.getReplicatedLog().snapshotCommit(); + 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); + } + } + } - // TODO: Not sure if we want to be this aggressive with trimming stuff - trimPersistentData(success.metadata().sequenceNr()); + private void switchBehavior(Supplier supplier){ + reusableBehaviorStateHolder.init(getCurrentBehavior()); - } else if (message instanceof SaveSnapshotFailure) { - SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; + setCurrentBehavior(supplier.get()); - LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString()); - LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:"); + handleBehaviorChange(reusableBehaviorStateHolder, getCurrentBehavior()); + } - context.getReplicatedLog().snapshotRollback(); + protected RaftActorSnapshotMessageSupport newRaftActorSnapshotMessageSupport() { + return new RaftActorSnapshotMessageSupport(context, currentBehavior, + getRaftActorSnapshotCohort()); + } - LOG.info("Replicated Log rollbacked. Snapshot will be attempted in the next cycle." + - "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", - context.getReplicatedLog().getSnapshotIndex(), - context.getReplicatedLog().getSnapshotTerm(), - context.getReplicatedLog().size()); + private void onGetOnDemandRaftStats() { + // Debugging message to retrieve raft stats. + + OnDemandRaftState.Builder builder = OnDemandRaftState.builder() + .commitIndex(context.getCommitIndex()) + .currentTerm(context.getTermInformation().getCurrentTerm()) + .inMemoryJournalDataSize(replicatedLog().dataSize()) + .inMemoryJournalLogSize(replicatedLog().size()) + .isSnapshotCaptureInitiated(context.getSnapshotManager().isCapturing()) + .lastApplied(context.getLastApplied()) + .lastIndex(replicatedLog().lastIndex()) + .lastTerm(replicatedLog().lastTerm()) + .leader(getLeaderId()) + .raftState(currentBehavior.state().toString()) + .replicatedToAllIndex(currentBehavior.getReplicatedToAllIndex()) + .snapshotIndex(replicatedLog().getSnapshotIndex()) + .snapshotTerm(replicatedLog().getSnapshotTerm()) + .votedFor(context.getTermInformation().getVotedFor()) + .peerAddresses(new HashMap<>(context.getPeerAddresses())); + + ReplicatedLogEntry lastLogEntry = getLastLogEntry(); + if (lastLogEntry != null) { + builder.lastLogIndex(lastLogEntry.getIndex()); + builder.lastLogTerm(lastLogEntry.getTerm()); + } - } else if (message instanceof AddRaftPeer){ + if(getCurrentBehavior() instanceof AbstractLeader) { + AbstractLeader leader = (AbstractLeader)getCurrentBehavior(); + Collection followerIds = leader.getFollowerIds(); + List followerInfoList = Lists.newArrayListWithCapacity(followerIds.size()); + for(String id: followerIds) { + final FollowerLogInformation info = leader.getFollower(id); + followerInfoList.add(new FollowerInfo(id, info.getNextIndex(), info.getMatchIndex(), + info.isFollowerActive(), DurationFormatUtils.formatDurationHMS(info.timeSinceLastActivity()))); + } - // FIXME : Do not add raft peers like this. - // When adding a new Peer we have to ensure that the a majority of - // the peers know about the new Peer. Doing it this way may cause - // a situation where multiple Leaders may emerge - AddRaftPeer arp = (AddRaftPeer)message; - context.addToPeers(arp.getName(), arp.getAddress()); + builder.followerInfoList(followerInfoList); + } - } else if (message instanceof RemoveRaftPeer){ + sender().tell(builder.build(), self()); - RemoveRaftPeer rrp = (RemoveRaftPeer)message; - context.removePeer(rrp.getName()); + } - } else if (message instanceof CaptureSnapshot) { - LOG.info("CaptureSnapshot received by actor"); - CaptureSnapshot cs = (CaptureSnapshot)message; - captureSnapshot = cs; - createSnapshot(); + private void handleBehaviorChange(BehaviorStateHolder oldBehaviorState, RaftActorBehavior currentBehavior) { + RaftActorBehavior oldBehavior = oldBehaviorState.getBehavior(); - } else if (message instanceof CaptureSnapshotReply){ - LOG.info("CaptureSnapshotReply received by actor"); - CaptureSnapshotReply csr = (CaptureSnapshotReply) message; + if (oldBehavior != currentBehavior){ + onStateChanged(); + } - ByteString stateInBytes = csr.getSnapshot(); - LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size()); - handleCaptureSnapshotReply(stateInBytes); + String lastValidLeaderId = oldBehavior == null ? null : oldBehaviorState.getLastValidLeaderId(); + String oldBehaviorStateName = oldBehavior == null ? null : oldBehavior.state().name(); - } else { - if (!(message instanceof AppendEntriesMessages.AppendEntries) - && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { - if(LOG.isDebugEnabled()) { - LOG.debug("onReceiveCommand: message:" + message.getClass()); - } + // it can happen that the state has not changed but the leader has changed. + Optional roleChangeNotifier = getRoleChangeNotifier(); + if(!Objects.equal(lastValidLeaderId, currentBehavior.getLeaderId()) || + oldBehaviorState.getLeaderPayloadVersion() != currentBehavior.getLeaderPayloadVersion()) { + if(roleChangeNotifier.isPresent()) { + roleChangeNotifier.get().tell(newLeaderStateChanged(getId(), currentBehavior.getLeaderId(), + currentBehavior.getLeaderPayloadVersion()), getSelf()); } - RaftState state = - currentBehavior.handleMessage(getSender(), message); - RaftActorBehavior oldBehavior = currentBehavior; - currentBehavior = switchBehavior(state); - if(oldBehavior != currentBehavior){ - onStateChanged(); - } + onLeaderChanged(lastValidLeaderId, currentBehavior.getLeaderId()); + } - onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId()); + if (roleChangeNotifier.isPresent() && + (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) { + roleChangeNotifier.get().tell(new RoleChanged(getId(), oldBehaviorStateName , + currentBehavior.state().name()), getSelf()); } } - public java.util.Set getPeers() { - return context.getPeerAddresses().keySet(); + protected LeaderStateChanged newLeaderStateChanged(String memberId, String leaderId, short leaderPayloadVersion) { + return new LeaderStateChanged(memberId, leaderId, leaderPayloadVersion); } - protected String getReplicatedLogState() { - return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex() - + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm() - + ", im-mem journal size=" + context.getReplicatedLog().size(); + @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. @@ -317,25 +360,62 @@ public abstract class RaftActor extends UntypedPersistentActor { * @param identifier * @param data */ - protected void persistData(ActorRef clientActor, String identifier, - Payload data) { + protected void persistData(final ActorRef clientActor, final String identifier, + final Payload data) { ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry( context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); if(LOG.isDebugEnabled()) { - LOG.debug("Persist data {}", replicatedLogEntry); + LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry); } - replicatedLog - .appendAndPersist(clientActor, identifier, replicatedLogEntry); + final RaftActorContext raftContext = getRaftActorContext(); + + replicatedLog().appendAndPersist(replicatedLogEntry, new Procedure() { + @Override + public void apply(ReplicatedLogEntry replicatedLogEntry) throws Exception { + if (!hasFollowers()){ + // Increment the Commit Index and the Last Applied values + raftContext.setCommitIndex(replicatedLogEntry.getIndex()); + raftContext.setLastApplied(replicatedLogEntry.getIndex()); + + // 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()); + + } else if (clientActor != null) { + context.getReplicatedLog().captureSnapshotIfReady(replicatedLogEntry); + + // Send message for replication + currentBehavior.handleMessage(getSelf(), + new Replicate(clientActor, identifier, replicatedLogEntry)); + } + } + }); + } + + private ReplicatedLog replicatedLog() { + return context.getReplicatedLog(); } protected String getId() { return context.getId(); } + @VisibleForTesting + void setCurrentBehavior(RaftActorBehavior behavior) { + currentBehavior.setDelegate(behavior); + } + + protected RaftActorBehavior getCurrentBehavior() { + return currentBehavior.getDelegate(); + } + /** * Derived actors can call the isLeader method to check if the current * RaftActor is the Leader or not @@ -376,7 +456,7 @@ public abstract class RaftActor extends UntypedPersistentActor { } protected ReplicatedLogEntry getLastLogEntry() { - return replicatedLog.last(); + return replicatedLog().last(); } protected Long getCurrentTerm(){ @@ -391,6 +471,49 @@ public abstract class RaftActor extends UntypedPersistentActor { return context.getLastApplied(); } + protected RaftActorContext getRaftActorContext() { + return context; + } + + protected void updateConfigParams(ConfigParams configParams) { + context.setConfigParams(configParams); + } + + public final DataPersistenceProvider persistence() { + return delegatingPersistenceProvider.getDelegate(); + } + + public void setPersistence(DataPersistenceProvider provider) { + delegatingPersistenceProvider.setDelegate(provider); + } + + protected void setPersistence(boolean persistent) { + if(persistent) { + setPersistence(new PersistentDataProvider(this)); + } else { + setPersistence(new NonPersistentDataProvider() { + /** + * The way snapshotting works is, + *
      + *
    1. RaftActor calls createSnapshot on the Shard + *
    2. Shard sends a CaptureSnapshotReply and RaftActor then calls saveSnapshot + *
    3. When saveSnapshot is invoked on the akka-persistence API it uses the SnapshotStore to save + * the snapshot. The SnapshotStore sends SaveSnapshotSuccess or SaveSnapshotFailure. When the + * RaftActor gets SaveSnapshot success it commits the snapshot to the in-memory journal. This + * commitSnapshot is mimicking what is done in SaveSnapshotSuccess. + *
    + */ + @Override + public void saveSnapshot(Object o) { + // 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(RaftActorSnapshotMessageSupport.COMMIT_SNAPSHOT, self()); + } + }); + } + } + /** * setPeerAddress sets the address of a known peer at a later time. *

    @@ -409,8 +532,6 @@ public abstract class RaftActor extends UntypedPersistentActor { context.setPeerAddress(peerId, peerAddress); } - - /** * The applyState method will be called by the RaftActor when some data * needs to be applied to the actor's state @@ -432,27 +553,21 @@ public abstract class RaftActor extends UntypedPersistentActor { Object data); /** - * 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 RaftActorRecoveryCohort to participate in persistence recovery. */ - protected abstract void createSnapshot(); + @Nonnull + protected abstract RaftActorRecoveryCohort getRaftActorRecoveryCohort(); /** - * This method will be called by the RaftActor during recovery to - * reconstruct the state of the actor. - *

    - * This method may also 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 snapshot A snapshot of the state of the actor + * This method is called when recovery is complete. */ - protected abstract void applySnapshot(ByteString snapshot); + protected abstract void onRecoveryComplete(); + + /** + * Returns the RaftActorSnapshotCohort to participate in persistence recovery. + */ + @Nonnull + protected abstract RaftActorSnapshotCohort getRaftActorSnapshotCohort(); /** * This method will be called by the RaftActor when the state of the @@ -461,50 +576,13 @@ public abstract class RaftActor extends UntypedPersistentActor { */ protected abstract void onStateChanged(); - protected void onLeaderChanged(String oldLeader, String newLeader){}; - - private RaftActorBehavior switchBehavior(RaftState state) { - if (currentBehavior != null) { - if (currentBehavior.state() == state) { - return currentBehavior; - } - LOG.info("Switching from state " + currentBehavior.state() + " to " - + state); - - try { - currentBehavior.close(); - } catch (Exception e) { - LOG.error(e, - "Failed to close behavior : " + currentBehavior.state()); - } - - } else { - LOG.info("Switching behavior to " + state); - } - RaftActorBehavior behavior = null; - if (state == RaftState.Candidate) { - behavior = new Candidate(context); - } else if (state == RaftState.Follower) { - behavior = new Follower(context); - } else { - behavior = new Leader(context); - } - - - - return behavior; - } - - private void trimPersistentData(long sequenceNumber) { - // Trim akka snapshots - // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied - // For now guessing that it is ANDed. - deleteSnapshots(new SnapshotSelectionCriteria( - sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); + /** + * Notifier Actor for this RaftActor to notify when a role change happens + * @return ActorRef - ActorRef of the notifier or Optional.absent if none. + */ + protected abstract Optional getRoleChangeNotifier(); - // Trim akka journal - deleteMessages(sequenceNumber); - } + protected void onLeaderChanged(String oldLeader, String newLeader){}; private String getLeaderAddress(){ if(isLeader()){ @@ -516,140 +594,39 @@ public abstract class RaftActor extends UntypedPersistentActor { } String peerAddress = context.getPeerAddress(leaderId); if(LOG.isDebugEnabled()) { - LOG.debug("getLeaderAddress leaderId = " + leaderId + " peerAddress = " - + peerAddress); + LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}", + persistenceId(), leaderId, peerAddress); } return peerAddress; } - private void handleCaptureSnapshotReply(ByteString stateInBytes) { - // create a snapshot object from the state provided and save it - // when snapshot is saved async, SaveSnapshotSuccess is raised. - - Snapshot sn = Snapshot.create(stateInBytes.toByteArray(), - context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), - captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), - captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - - saveSnapshot(sn); - - LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); - - //be greedy and remove entries from in-mem journal which are in the snapshot - // and update snapshotIndex and snapshotTerm without waiting for the success, - - context.getReplicatedLog().snapshotPreCommit(stateInBytes, - captureSnapshot.getLastAppliedIndex(), - captureSnapshot.getLastAppliedTerm()); - - LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + - "and term:{}", captureSnapshot.getLastAppliedIndex(), - captureSnapshot.getLastAppliedTerm()); - - captureSnapshot = null; - hasSnapshotCaptureInitiated = false; + protected boolean hasFollowers(){ + return getRaftActorContext().hasFollowers(); } + private void captureSnapshot() { + SnapshotManager snapshotManager = context.getSnapshotManager(); - private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { - - public ReplicatedLogImpl(Snapshot snapshot) { - super(ByteString.copyFrom(snapshot.getState()), - snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), - snapshot.getUnAppliedEntries()); - } - - public ReplicatedLogImpl() { - super(); - } - - @Override public void removeFromAndPersist(long logEntryIndex) { - int adjustedIndex = adjustedIndex(logEntryIndex); - - if (adjustedIndex < 0) { - return; - } - - // FIXME: Maybe this should be done after the command is saved - journal.subList(adjustedIndex , journal.size()).clear(); - - persist(new DeleteEntries(adjustedIndex), new Procedure(){ - - @Override public void apply(DeleteEntries param) - throws Exception { - //FIXME : Doing nothing for now - } - }); - } - - @Override public void appendAndPersist( - final ReplicatedLogEntry replicatedLogEntry) { - appendAndPersist(null, null, replicatedLogEntry); - } + if(!snapshotManager.isCapturing()) { + LOG.debug("Take a snapshot of current state. lastReplicatedLog is {} and replicatedToAllIndex is {}", + replicatedLog().last(), currentBehavior.getReplicatedToAllIndex()); - public void appendAndPersist(final ActorRef clientActor, - final String identifier, - final ReplicatedLogEntry replicatedLogEntry) { - context.getLogger().debug( - "Append log entry and persist {} ", replicatedLogEntry); - // FIXME : By adding the replicated log entry to the in-memory journal we are not truly ensuring durability of the logs - journal.add(replicatedLogEntry); - - // When persisting events with persist it is guaranteed that the - // persistent actor will not receive further commands between the - // persist call and the execution(s) of the associated event - // handler. This also holds for multiple persist calls in context - // of a single command. - persist(replicatedLogEntry, - new Procedure() { - public void apply(ReplicatedLogEntry evt) throws Exception { - // when a snaphsot is being taken, captureSnapshot != null - if (hasSnapshotCaptureInitiated == false && - journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) { - - LOG.info("Initiating Snapshot Capture.."); - long lastAppliedIndex = -1; - long lastAppliedTerm = -1; - - ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied()); - if (lastAppliedEntry != null) { - lastAppliedIndex = lastAppliedEntry.getIndex(); - lastAppliedTerm = lastAppliedEntry.getTerm(); - } - - if(LOG.isDebugEnabled()) { - LOG.debug("Snapshot Capture logSize: {}", journal.size()); - LOG.debug("Snapshot Capture lastApplied:{} ", - context.getLastApplied()); - LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex); - LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm); - } - - // send a CaptureSnapshot to self to make the expensive operation async. - getSelf().tell(new CaptureSnapshot( - lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm), - null); - hasSnapshotCaptureInitiated = true; - } - // Send message for replication - if (clientActor != null) { - currentBehavior.handleMessage(getSelf(), - new Replicate(clientActor, identifier, - replicatedLogEntry) - ); - } - } - } - ); + snapshotManager.capture(replicatedLog().last(), currentBehavior.getReplicatedToAllIndex()); } - } - private static class DeleteEntries implements Serializable { + /** + * @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 final int fromIndex; - public DeleteEntries(int fromIndex) { this.fromIndex = fromIndex; } @@ -659,13 +636,21 @@ public abstract class RaftActor extends UntypedPersistentActor { } } + /** + * @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 final long currentTerm; + private final String votedFor; - private class ElectionTermImpl implements ElectionTerm { - /** - * Identifier of the actor whose election term information this is - */ - private long currentTerm = 0; - private String votedFor = null; + public UpdateElectionTerm(long currentTerm, String votedFor) { + this.currentTerm = currentTerm; + this.votedFor = votedFor; + } public long getCurrentTerm() { return currentTerm; @@ -674,45 +659,52 @@ public abstract class RaftActor extends UntypedPersistentActor { public String getVotedFor() { return votedFor; } + } - @Override public void update(long currentTerm, String votedFor) { - if(LOG.isDebugEnabled()) { - LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); + private static class BehaviorStateHolder { + private RaftActorBehavior behavior; + private String lastValidLeaderId; + private short leaderPayloadVersion; + + void init(RaftActorBehavior behavior) { + this.behavior = behavior; + this.leaderPayloadVersion = behavior != null ? behavior.getLeaderPayloadVersion() : -1; + + String behaviorLeaderId = behavior != null ? behavior.getLeaderId() : null; + if(behaviorLeaderId != null) { + this.lastValidLeaderId = behaviorLeaderId; } - this.currentTerm = currentTerm; - this.votedFor = votedFor; } - @Override - public void updateAndPersist(long currentTerm, String votedFor){ - update(currentTerm, votedFor); - // FIXME : Maybe first persist then update the state - persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure(){ + RaftActorBehavior getBehavior() { + return behavior; + } - @Override public void apply(UpdateElectionTerm param) - throws Exception { + String getLastValidLeaderId() { + return lastValidLeaderId; + } - } - }); + short getLeaderPayloadVersion() { + return leaderPayloadVersion; } } - private static class UpdateElectionTerm implements Serializable { - private final long currentTerm; - private final String votedFor; + private class SwitchBehaviorSupplier implements Supplier { + private Object message; + private ActorRef sender; - public UpdateElectionTerm(long currentTerm, String votedFor) { - this.currentTerm = currentTerm; - this.votedFor = votedFor; - } - - public long getCurrentTerm() { - return currentTerm; + public SwitchBehaviorSupplier handleMessage(ActorRef sender, Object message){ + this.sender = sender; + this.message = message; + return this; } - public String getVotedFor() { - return votedFor; + @Override + public RaftActorBehavior get() { + if(this.message instanceof SwitchBehavior){ + return ((SwitchBehavior) message).getNewState().createBehavior(getRaftActorContext()); + } + return currentBehavior.handleMessage(sender, message); } } - }