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=d647475e4d53319e1638c0e4c03fe4dfcd58a891;hp=8270f2949a67cc9fc00f5180dce41872ca6a8a47;hb=a469dbcec569cc972df0cd57cf725a2173d2604a;hpb=592ff3cd9e1b80d4cbd9ca45ed4a2371377a7622 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 8270f2949a..d647475e4d 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 @@ -18,30 +18,32 @@ 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.Optional; +import com.google.common.base.Stopwatch; import com.google.protobuf.ByteString; +import java.io.Serializable; +import java.util.Map; +import org.opendaylight.controller.cluster.DataPersistenceProvider; +import org.opendaylight.controller.cluster.common.actor.AbstractUntypedPersistentActor; +import org.opendaylight.controller.cluster.notifications.RoleChanged; +import org.opendaylight.controller.cluster.raft.base.messages.ApplyLogEntries; import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.base.messages.ApplyState; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshotReply; import org.opendaylight.controller.cluster.raft.base.messages.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.SendInstallSnapshot; +import org.opendaylight.controller.cluster.raft.behaviors.AbstractRaftActorBehavior; 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.protobuff.client.messages.Payload; import org.opendaylight.controller.protobuff.messages.cluster.raft.AppendEntriesMessages; -import java.io.Serializable; -import java.util.Map; - /** * RaftActor encapsulates a state machine that needs to be kept synchronized * in a cluster. It implements the RAFT algorithm as described in the paper @@ -82,7 +84,7 @@ import java.util.Map; *
  • when a snapshot should be saved
  • * */ -public abstract class RaftActor extends UntypedPersistentActor { +public abstract class RaftActor extends AbstractUntypedPersistentActor { protected final LoggingAdapter LOG = Logging.getLogger(getContext().system(), this); @@ -96,7 +98,7 @@ public abstract class RaftActor extends UntypedPersistentActor { * 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 RaftActorContext context; /** * The in-memory journal @@ -107,6 +109,10 @@ public abstract class RaftActor extends UntypedPersistentActor { private volatile boolean hasSnapshotCaptureInitiated = false; + private Stopwatch recoveryTimer; + + private int currentRecoveryBatchCount; + public RaftActor(String id, Map peerAddresses) { this(id, peerAddresses, Optional.absent()); } @@ -121,60 +127,156 @@ public abstract class RaftActor extends UntypedPersistentActor { LOG); } - @Override public void onReceiveRecover(Object message) { - if (message instanceof SnapshotOffer) { - LOG.info("SnapshotOffer called.."); - SnapshotOffer offer = (SnapshotOffer) message; - Snapshot snapshot = (Snapshot) offer.snapshot(); + private void initRecoveryTimer() { + if(recoveryTimer == null) { + recoveryTimer = new Stopwatch(); + recoveryTimer.start(); + } + } - // 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); + @Override + public void preStart() throws Exception { + LOG.info("Starting recovery for {} with journal batch size {}", persistenceId(), + context.getConfigParams().getJournalRecoveryLogBatchSize()); - context.setReplicatedLog(replicatedLog); - context.setLastApplied(snapshot.getLastAppliedIndex()); - context.setCommitIndex(snapshot.getLastAppliedIndex()); + super.preStart(); + } - LOG.info("Applied snapshot to replicatedLog. " + - "snapshotIndex={}, snapshotTerm={}, journal-size={}", - replicatedLog.snapshotIndex, replicatedLog.snapshotTerm, - replicatedLog.size() - ); + @Override + public void handleRecover(Object message) { + if(persistence().isRecoveryApplicable()) { + if (message instanceof SnapshotOffer) { + onRecoveredSnapshot((SnapshotOffer) message); + } else if (message instanceof ReplicatedLogEntry) { + onRecoveredJournalLogEntry((ReplicatedLogEntry) message); + } else if (message instanceof ApplyLogEntries) { + onRecoveredApplyLogEntries((ApplyLogEntries) message); + } else if (message instanceof DeleteEntries) { + replicatedLog.removeFrom(((DeleteEntries) message).getFromIndex()); + } else if (message instanceof UpdateElectionTerm) { + context.getTermInformation().update(((UpdateElectionTerm) message).getCurrentTerm(), + ((UpdateElectionTerm) message).getVotedFor()); + } else if (message instanceof RecoveryCompleted) { + onRecoveryCompletedMessage(); + } + } else { + if (message instanceof RecoveryCompleted) { + // Delete all the messages from the akka journal so that we do not end up with consistency issues + // Note I am not using the dataPersistenceProvider and directly using the akka api here + deleteMessages(lastSequenceNr()); - // Apply the snapshot to the actors state - applySnapshot(ByteString.copyFrom(snapshot.getState())); + // Delete all the akka snapshots as they will not be needed + deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue())); - } else if (message instanceof ReplicatedLogEntry) { - ReplicatedLogEntry logEntry = (ReplicatedLogEntry) message; - - // Apply State immediately - replicatedLog.append(logEntry); - applyState(null, "recovery", logEntry.getData()); - context.setLastApplied(logEntry.getIndex()); - context.setCommitIndex(logEntry.getIndex()); - - } 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(); + onRecoveryComplete(); + + RaftActorBehavior oldBehavior = currentBehavior; + currentBehavior = new Follower(context); + handleBehaviorChange(oldBehavior, currentBehavior); + } } } - @Override public void onReceiveCommand(Object message) { + private void onRecoveredSnapshot(SnapshotOffer offer) { + if(LOG.isDebugEnabled()) { + LOG.debug("SnapshotOffer called.."); + } + + initRecoveryTimer(); + + Snapshot snapshot = (Snapshot) offer.snapshot(); + + // 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); + + context.setReplicatedLog(replicatedLog); + context.setLastApplied(snapshot.getLastAppliedIndex()); + context.setCommitIndex(snapshot.getLastAppliedIndex()); + + Stopwatch timer = new Stopwatch(); + timer.start(); + + // Apply the snapshot to the actors state + applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState())); + + timer.stop(); + LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" + + replicatedLog.size(), persistenceId(), timer.toString(), + replicatedLog.snapshotIndex, replicatedLog.snapshotTerm); + } + + private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) { + if(LOG.isDebugEnabled()) { + LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex()); + } + + replicatedLog.append(logEntry); + } + + private void onRecoveredApplyLogEntries(ApplyLogEntries ale) { + if(LOG.isDebugEnabled()) { + LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}", + context.getLastApplied() + 1, ale.getToIndex()); + } + + for (long i = context.getLastApplied() + 1; i <= ale.getToIndex(); i++) { + batchRecoveredLogEntry(replicatedLog.get(i)); + } + + context.setLastApplied(ale.getToIndex()); + context.setCommitIndex(ale.getToIndex()); + } + + private void batchRecoveredLogEntry(ReplicatedLogEntry logEntry) { + initRecoveryTimer(); + + int batchSize = context.getConfigParams().getJournalRecoveryLogBatchSize(); + if(currentRecoveryBatchCount == 0) { + startLogRecoveryBatch(batchSize); + } + + appendRecoveredLogEntry(logEntry.getData()); + + if(++currentRecoveryBatchCount >= batchSize) { + endCurrentLogRecoveryBatch(); + } + } + + private void endCurrentLogRecoveryBatch() { + applyCurrentLogRecoveryBatch(); + currentRecoveryBatchCount = 0; + } + + private void onRecoveryCompletedMessage() { + if(currentRecoveryBatchCount > 0) { + endCurrentLogRecoveryBatch(); + } + + onRecoveryComplete(); + + String recoveryTime = ""; + if(recoveryTimer != null) { + recoveryTimer.stop(); + recoveryTime = " in " + recoveryTimer.toString(); + recoveryTimer = null; + } + + LOG.info( + "Recovery completed" + recoveryTime + " - Switching actor to Follower - " + + "Persistence Id = " + persistenceId() + + " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " + + "journal-size={}", + replicatedLog.lastIndex(), replicatedLog.snapshotIndex, + replicatedLog.snapshotTerm, replicatedLog.size()); + + RaftActorBehavior oldBehavior = currentBehavior; + currentBehavior = new Follower(context); + handleBehaviorChange(oldBehavior, currentBehavior); + } + + @Override public void handleCommand(Object message) { if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; @@ -187,6 +289,17 @@ public abstract class RaftActor extends UntypedPersistentActor { applyState(applyState.getClientActor(), applyState.getIdentifier(), applyState.getReplicatedLogEntry().getData()); + } else if (message instanceof ApplyLogEntries){ + ApplyLogEntries ale = (ApplyLogEntries) message; + if(LOG.isDebugEnabled()) { + LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex()); + } + persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure() { + @Override + public void apply(ApplyLogEntries param) throws Exception { + } + }); + } else if(message instanceof ApplySnapshot ) { Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); @@ -213,10 +326,9 @@ public abstract class RaftActor extends UntypedPersistentActor { SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; LOG.info("SaveSnapshotSuccess received for snapshot"); - context.getReplicatedLog().snapshotCommit(); + long sequenceNumber = success.metadata().sequenceNr(); - // TODO: Not sure if we want to be this aggressive with trimming stuff - trimPersistentData(success.metadata().sequenceNr()); + commitSnapshot(sequenceNumber); } else if (message instanceof SaveSnapshotFailure) { SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; @@ -232,20 +344,6 @@ public abstract class RaftActor extends UntypedPersistentActor { context.getReplicatedLog().getSnapshotTerm(), context.getReplicatedLog().size()); - } else if (message instanceof AddRaftPeer){ - - // 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()); - - } else if (message instanceof RemoveRaftPeer){ - - RemoveRaftPeer rrp = (RemoveRaftPeer)message; - context.removePeer(rrp.getName()); - } else if (message instanceof CaptureSnapshot) { LOG.info("CaptureSnapshot received by actor"); CaptureSnapshot cs = (CaptureSnapshot)message; @@ -264,33 +362,33 @@ public abstract class RaftActor extends UntypedPersistentActor { if (!(message instanceof AppendEntriesMessages.AppendEntries) && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { if(LOG.isDebugEnabled()) { - LOG.debug("onReceiveCommand: message:" + message.getClass()); + LOG.debug("onReceiveCommand: message: {}", message.getClass()); } } - RaftState state = - currentBehavior.handleMessage(getSender(), message); RaftActorBehavior oldBehavior = currentBehavior; - currentBehavior = switchBehavior(state); - if(oldBehavior != currentBehavior){ - onStateChanged(); - } + currentBehavior = currentBehavior.handleMessage(getSender(), message); - onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId()); + handleBehaviorChange(oldBehavior, currentBehavior); } } - public java.util.Set getPeers() { - return context.getPeerAddresses().keySet(); - } + private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) { + if (oldBehavior != currentBehavior){ + onStateChanged(); + } + if (oldBehavior != null) { + // it can happen that the state has not changed but the leader has changed. + onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId()); - protected String getReplicatedLogState() { - return "snapshotIndex=" + context.getReplicatedLog().getSnapshotIndex() - + ", snapshotTerm=" + context.getReplicatedLog().getSnapshotTerm() - + ", im-mem journal size=" + context.getReplicatedLog().size(); + if (getRoleChangeNotifier().isPresent() && oldBehavior.state() != currentBehavior.state()) { + // we do not want to notify when the behavior/role is set for the first time (i.e follower) + getRoleChangeNotifier().get().tell(new RoleChanged(getId(), oldBehavior.state().name(), + currentBehavior.state().name()), getSelf()); + } + } } - /** * When a derived RaftActor needs to persist something it must call * persistData. @@ -373,6 +471,10 @@ public abstract class RaftActor extends UntypedPersistentActor { return context.getLastApplied(); } + protected RaftActorContext getRaftActorContext() { + return context; + } + /** * setPeerAddress sets the address of a known peer at a later time. *

    @@ -391,7 +493,12 @@ public abstract class RaftActor extends UntypedPersistentActor { context.setPeerAddress(peerId, peerAddress); } + protected void commitSnapshot(long sequenceNumber) { + context.getReplicatedLog().snapshotCommit(); + // TODO: Not sure if we want to be this aggressive with trimming stuff + trimPersistentData(sequenceNumber); + } /** * The applyState method will be called by the RaftActor when some data @@ -413,6 +520,38 @@ public abstract class RaftActor extends UntypedPersistentActor { protected abstract void applyState(ActorRef clientActor, String identifier, Object data); + /** + * This method is called during recovery at the start of a batch of state entries. Derived + * classes should perform any initialization needed to start a batch. + */ + protected abstract void startLogRecoveryBatch(int maxBatchSize); + + /** + * This method is called during recovery to append state data to the current batch. This method + * is called 1 or more times after {@link #startLogRecoveryBatch}. + * + * @param data the state data + */ + protected abstract void appendRecoveredLogEntry(Payload data); + + /** + * This method is called during recovery to reconstruct the state of the actor. + * + * @param snapshot A snapshot of the state of the actor + */ + protected abstract void applyRecoverySnapshot(ByteString snapshot); + + /** + * This method is called during recovery at the end of a batch to apply the current batched + * log entries. This method is called after {@link #appendRecoveredLogEntry}. + */ + protected abstract void applyCurrentLogRecoveryBatch(); + + /** + * This method is called when recovery is complete. + */ + 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. @@ -425,10 +564,7 @@ public abstract class RaftActor extends UntypedPersistentActor { protected abstract void createSnapshot(); /** - * 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 + * 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 * @@ -443,49 +579,25 @@ 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); - } - + protected abstract DataPersistenceProvider persistence(); + /** + * 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(); - return behavior; - } + protected void onLeaderChanged(String oldLeader, String newLeader){}; 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( + persistence().deleteSnapshots(new SnapshotSelectionCriteria( sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); // Trim akka journal - deleteMessages(sequenceNumber); + persistence().deleteMessages(sequenceNumber); } private String getLeaderAddress(){ @@ -498,8 +610,8 @@ 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 = {}", + leaderId, peerAddress); } return peerAddress; @@ -514,14 +626,14 @@ public abstract class RaftActor extends UntypedPersistentActor { captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - saveSnapshot(sn); + persistence().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, + context.getReplicatedLog().snapshotPreCommit( captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); @@ -529,16 +641,19 @@ public abstract class RaftActor extends UntypedPersistentActor { "and term:{}", captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); + if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) { + // this would be call straight to the leader and won't initiate in serialization + currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot(stateInBytes)); + } + captureSnapshot = null; hasSnapshotCaptureInitiated = false; } - private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { public ReplicatedLogImpl(Snapshot snapshot) { - super(ByteString.copyFrom(snapshot.getState()), - snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), + super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), snapshot.getUnAppliedEntries()); } @@ -556,7 +671,7 @@ public abstract class RaftActor extends UntypedPersistentActor { // FIXME: Maybe this should be done after the command is saved journal.subList(adjustedIndex , journal.size()).clear(); - persist(new DeleteEntries(adjustedIndex), new Procedure(){ + persistence().persist(new DeleteEntries(adjustedIndex), new Procedure(){ @Override public void apply(DeleteEntries param) throws Exception { @@ -573,8 +688,11 @@ public abstract class RaftActor extends UntypedPersistentActor { public void appendAndPersist(final ActorRef clientActor, final String identifier, final ReplicatedLogEntry replicatedLogEntry) { - context.getLogger().debug( - "Append log entry and persist {} ", replicatedLogEntry); + + if(LOG.isDebugEnabled()) { + LOG.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); @@ -583,8 +701,9 @@ public abstract class RaftActor extends UntypedPersistentActor { // 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, + persistence().persist(replicatedLogEntry, new Procedure() { + @Override public void apply(ReplicatedLogEntry evt) throws Exception { // when a snaphsot is being taken, captureSnapshot != null if (hasSnapshotCaptureInitiated == false && @@ -628,10 +747,10 @@ public abstract class RaftActor extends UntypedPersistentActor { } - private static class DeleteEntries implements Serializable { + static class DeleteEntries implements Serializable { + private static final long serialVersionUID = 1L; private final int fromIndex; - public DeleteEntries(int fromIndex) { this.fromIndex = fromIndex; } @@ -649,10 +768,12 @@ public abstract class RaftActor extends UntypedPersistentActor { private long currentTerm = 0; private String votedFor = null; + @Override public long getCurrentTerm() { return currentTerm; } + @Override public String getVotedFor() { return votedFor; } @@ -669,7 +790,7 @@ public abstract class RaftActor extends UntypedPersistentActor { 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(){ + persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure(){ @Override public void apply(UpdateElectionTerm param) throws Exception { @@ -679,7 +800,8 @@ public abstract class RaftActor extends UntypedPersistentActor { } } - private static class UpdateElectionTerm implements Serializable { + static class UpdateElectionTerm implements Serializable { + private static final long serialVersionUID = 1L; private final long currentTerm; private final String votedFor; @@ -697,4 +819,38 @@ public abstract class RaftActor extends UntypedPersistentActor { } } + protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider { + + public NonPersistentRaftDataProvider(){ + + } + + /** + * 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. + *
    + * @param o + */ + @Override + public void saveSnapshot(Object o) { + // Make saving Snapshot successful + commitSnapshot(-1L); + } + } + + @VisibleForTesting + void setCurrentBehavior(AbstractRaftActorBehavior behavior) { + currentBehavior = behavior; + } + + protected RaftActorBehavior getCurrentBehavior() { + return currentBehavior; + } + }