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=988789b4011e4f3ba9e9e9abe6b0c3009704c810;hb=3a71a222b896b9d07e638af62300180799bdac67;hp=0ff2341c708431d8da540d625638c9d70bb3dd11;hpb=d255fdd0b14660a22ff63771d954ac3fe5d0cb7e;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 0ff2341c70..3b84692077 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,23 +18,31 @@ import akka.persistence.SaveSnapshotFailure; import akka.persistence.SaveSnapshotSuccess; import akka.persistence.SnapshotOffer; import akka.persistence.SnapshotSelectionCriteria; -import akka.persistence.UntypedPersistentActor; -import org.opendaylight.controller.cluster.raft.behaviors.Candidate; +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.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.internal.messages.ApplySnapshot; -import org.opendaylight.controller.cluster.raft.client.messages.RemoveRaftPeer; -import org.opendaylight.controller.cluster.raft.internal.messages.ApplyState; -import org.opendaylight.controller.cluster.raft.internal.messages.Replicate; - -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; +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; /** * RaftActor encapsulates a state machine that needs to be kept synchronized @@ -76,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); @@ -90,99 +98,296 @@ 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 */ - private RaftActorContext context; + private final RaftActorContext context; /** * The in-memory journal */ private ReplicatedLogImpl replicatedLog = new ReplicatedLogImpl(); + private CaptureSnapshot captureSnapshot = null; + + private volatile boolean hasSnapshotCaptureInitiated = false; + + private Stopwatch recoveryTimer; + + private int currentRecoveryBatchCount; public RaftActor(String id, Map peerAddresses) { + this(id, peerAddresses, Optional.absent()); + } + + public RaftActor(String id, Map peerAddresses, + Optional configParams) { + context = new RaftActorContextImpl(this.getSelf(), - this.getContext(), - id, new ElectionTermImpl(), - -1, -1, replicatedLog, peerAddresses, LOG); + this.getContext(), id, new ElectionTermImpl(), + -1, -1, replicatedLog, peerAddresses, + (configParams.isPresent() ? configParams.get(): new DefaultConfigParamsImpl()), + LOG); } - @Override public void onReceiveRecover(Object message) { - if (message instanceof SnapshotOffer) { - 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()); - // Apply the snapshot to the actors state - applySnapshot(snapshot.getState()); + super.preStart(); + } - } else if (message instanceof ReplicatedLogEntry) { - replicatedLog.append((ReplicatedLogEntry) 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) { - LOG.debug( - "Last index in log : " + replicatedLog.lastIndex()); - currentBehavior = switchBehavior(RaftState.Follower); + @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()); + + // Delete all the akka snapshots as they will not be needed + deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue())); + + 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; - LOG.debug("Applying state for log index {} data {}", - applyState.getReplicatedLogEntry().getIndex(), - applyState.getReplicatedLogEntry().getData()); + if(LOG.isDebugEnabled()) { + LOG.debug("Applying state for log index {} data {}", + applyState.getReplicatedLogEntry().getIndex(), + applyState.getReplicatedLogEntry().getData()); + } 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 ) { - applySnapshot(((ApplySnapshot) message).getSnapshot()); + Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); + + if(LOG.isDebugEnabled()) { + LOG.debug("ApplySnapshot called on Follower Actor " + + "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(), + snapshot.getLastAppliedTerm() + ); + } + 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()); + } else if (message instanceof FindLeader) { getSender().tell( - new FindLeaderReply( - context.getPeerAddress(currentBehavior.getLeaderId())), + new FindLeaderReply(getLeaderAddress()), getSelf() ); + } else if (message instanceof SaveSnapshotSuccess) { SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; + LOG.info("SaveSnapshotSuccess received for snapshot"); + + 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; - // TODO: Handle failure in saving the snapshot - // Maybe do retries on failure + LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString()); + LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:"); - } else if (message instanceof AddRaftPeer){ + context.getReplicatedLog().snapshotRollback(); - // 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()); + 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()); - } else if (message instanceof RemoveRaftPeer){ + } else if (message instanceof CaptureSnapshot) { + LOG.info("CaptureSnapshot received by actor"); + CaptureSnapshot cs = (CaptureSnapshot)message; + captureSnapshot = cs; + createSnapshot(); - RemoveRaftPeer rrp = (RemoveRaftPeer)message; - context.removePeer(rrp.getName()); + } else if (message instanceof CaptureSnapshotReply){ + LOG.info("CaptureSnapshotReply received by actor"); + CaptureSnapshotReply csr = (CaptureSnapshotReply) message; + + ByteString stateInBytes = csr.getSnapshot(); + LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size()); + handleCaptureSnapshotReply(stateInBytes); } else { + if (!(message instanceof AppendEntriesMessages.AppendEntries) + && !(message instanceof AppendEntriesReply) && !(message instanceof SendHeartBeat)) { + if(LOG.isDebugEnabled()) { + LOG.debug("onReceiveCommand: message: {}", message.getClass()); + } + } + + RaftActorBehavior oldBehavior = currentBehavior; + currentBehavior = currentBehavior.handleMessage(getSender(), message); - RaftState state = - currentBehavior.handleMessage(getSender(), message); - currentBehavior = switchBehavior(state); + handleBehaviorChange(oldBehavior, currentBehavior); } } + 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()); + 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 @@ -193,13 +398,15 @@ public abstract class RaftActor extends UntypedPersistentActor { * @param data */ protected void persistData(ActorRef clientActor, String identifier, - Object data) { + Payload data) { ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry( context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); - LOG.debug("Persist data {}", replicatedLogEntry); + if(LOG.isDebugEnabled()) { + LOG.debug("Persist data {}", replicatedLogEntry); + } replicatedLog .appendAndPersist(clientActor, identifier, replicatedLogEntry); @@ -227,21 +434,71 @@ public abstract class RaftActor extends UntypedPersistentActor { * @return A reference to the leader if known, null otherwise */ protected ActorSelection getLeader(){ - String leaderId = currentBehavior.getLeaderId(); - if (leaderId == null) { + String leaderAddress = getLeaderAddress(); + + if(leaderAddress == null){ return null; } - String peerAddress = context.getPeerAddress(leaderId); - LOG.debug("getLeader leaderId = " + leaderId + " peerAddress = " - + peerAddress); - return context.actorSelection(peerAddress); + + return context.actorSelection(leaderAddress); + } + + /** + * + * @return the current leader's id + */ + protected String getLeaderId(){ + return currentBehavior.getLeaderId(); } protected RaftState getRaftState() { return currentBehavior.state(); } + protected ReplicatedLogEntry getLastLogEntry() { + return replicatedLog.last(); + } + + protected Long getCurrentTerm(){ + return context.getTermInformation().getCurrentTerm(); + } + + protected Long getCommitIndex(){ + return context.getCommitIndex(); + } + protected Long getLastApplied(){ + return context.getLastApplied(); + } + + protected RaftActorContext getRaftActorContext() { + return context; + } + + /** + * setPeerAddress sets the address of a known peer at a later time. + *

    + * This is to account for situations where a we know that a peer + * exists but we do not know an address up-front. This may also be used in + * situations where a known peer starts off in a different location and we + * need to change it's address + *

    + * Note that if the peerId does not match the list of peers passed to + * this actor during construction an IllegalStateException will be thrown. + * + * @param peerId + * @param peerAddress + */ + protected void setPeerAddress(String peerId, String peerAddress){ + 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 @@ -263,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. @@ -272,165 +561,127 @@ public abstract class RaftActor extends UntypedPersistentActor { * * @return The current state of the actor */ - protected abstract Object createSnapshot(); + 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 * * @param snapshot A snapshot of the state of the actor */ - protected abstract void applySnapshot(Object snapshot); + protected abstract void applySnapshot(ByteString snapshot); - 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()); - } + /** + * This method will be called by the RaftActor when the state of the + * RaftActor changes. The derived actor can then use methods like + * isLeader or getLeader to do something useful + */ + protected abstract void onStateChanged(); - } 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; - } + 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(); + + protected void onLeaderChanged(String oldLeader, String newLeader){}; private void trimPersistentData(long sequenceNumber) { - // Trim snapshots + // Trim akka snapshots // FIXME : Not sure how exactly the SnapshotSelectionCriteria is applied // For now guessing that it is ANDed. - deleteSnapshots(new SnapshotSelectionCriteria( - sequenceNumber - 100000, 43200000)); + persistence().deleteSnapshots(new SnapshotSelectionCriteria( + sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); - // Trim journal - deleteMessages(sequenceNumber); + // Trim akka journal + persistence().deleteMessages(sequenceNumber); } - - private class ReplicatedLogImpl implements ReplicatedLog { - private final List journal; - private final Object snapshot; - private long snapshotIndex = -1; - private long snapshotTerm = -1; - - public ReplicatedLogImpl(Snapshot snapshot) { - this.snapshot = snapshot.getState(); - this.snapshotIndex = snapshot.getLastAppliedIndex(); - this.snapshotTerm = snapshot.getLastAppliedTerm(); - - this.journal = new ArrayList<>(snapshot.getUnAppliedEntries()); + private String getLeaderAddress(){ + if(isLeader()){ + return getSelf().path().toString(); } - - public ReplicatedLogImpl() { - this.snapshot = null; - this.journal = new ArrayList<>(); + String leaderId = currentBehavior.getLeaderId(); + if (leaderId == null) { + return null; + } + String peerAddress = context.getPeerAddress(leaderId); + if(LOG.isDebugEnabled()) { + LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}", + leaderId, peerAddress); } - @Override public ReplicatedLogEntry get(long index) { - int adjustedIndex = adjustedIndex(index); + return peerAddress; + } - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return null; - } + private void handleCaptureSnapshotReply(ByteString stateInBytes) { + // create a snapshot object from the state provided and save it + // when snapshot is saved async, SaveSnapshotSuccess is raised. - return journal.get(adjustedIndex); - } + Snapshot sn = Snapshot.create(stateInBytes.toByteArray(), + context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), + captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), + captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - @Override public ReplicatedLogEntry last() { - if (journal.size() == 0) { - return null; - } - return get(journal.size() - 1); - } + persistence().saveSnapshot(sn); - @Override public long lastIndex() { - if (journal.size() == 0) { - return -1; - } + LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); - return last().getIndex(); - } + //be greedy and remove entries from in-mem journal which are in the snapshot + // and update snapshotIndex and snapshotTerm without waiting for the success, - @Override public long lastTerm() { - if (journal.size() == 0) { - return -1; - } + context.getReplicatedLog().snapshotPreCommit( + captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); - return last().getTerm(); - } + LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + + "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)); + } - @Override public void removeFrom(long index) { - int adjustedIndex = adjustedIndex(index); + captureSnapshot = null; + hasSnapshotCaptureInitiated = false; + } - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return; - } + private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { - journal.subList(adjustedIndex , journal.size()).clear(); + public ReplicatedLogImpl(Snapshot snapshot) { + super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), + snapshot.getUnAppliedEntries()); } + public ReplicatedLogImpl() { + super(); + } - @Override public void removeFromAndPersist(long index) { - int adjustedIndex = adjustedIndex(index); + @Override public void removeFromAndPersist(long logEntryIndex) { + int adjustedIndex = adjustedIndex(logEntryIndex); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { + 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(){ + persistence().persist(new DeleteEntries(adjustedIndex), new Procedure(){ @Override public void apply(DeleteEntries param) throws Exception { //FIXME : Doing nothing for now + dataSize = 0; + for(ReplicatedLogEntry entry : journal){ + dataSize += entry.size(); + } } }); - - - } - - @Override public void append( - final ReplicatedLogEntry replicatedLogEntry) { - journal.add(replicatedLogEntry); - } - - @Override public List getFrom(long index) { - int adjustedIndex = adjustedIndex(index); - - List entries = new ArrayList<>(100); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return entries; - } - - - for (int i = adjustedIndex; - i < journal.size(); i++) { - entries.add(journal.get(i)); - } - return entries; } @Override public void appendAndPersist( @@ -438,11 +689,19 @@ public abstract class RaftActor extends UntypedPersistentActor { appendAndPersist(null, null, replicatedLogEntry); } + @Override + public int dataSize() { + return dataSize; + } + 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); @@ -451,24 +710,43 @@ 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 { - // FIXME : Tentatively create a snapshot every hundred thousand entries. To be tuned. - if (size() > 100000) { - ReplicatedLogEntry lastAppliedEntry = - get(context.getLastApplied()); + dataSize += replicatedLogEntry.size(); + + long dataThreshold = Runtime.getRuntime().totalMemory() * + getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100; + + // when a snaphsot is being taken, captureSnapshot != null + if (hasSnapshotCaptureInitiated == false && + ( journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0 || + dataSize > dataThreshold)) { + + 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(); } - saveSnapshot(Snapshot.create(createSnapshot(), - getFrom(context.getLastApplied() + 1), - lastIndex(), lastTerm(), lastAppliedIndex, - lastAppliedTerm)); + 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) { @@ -482,82 +760,12 @@ public abstract class RaftActor extends UntypedPersistentActor { ); } - @Override public long size() { - return journal.size() + snapshotIndex + 1; - } - - @Override public boolean isPresent(long index) { - int adjustedIndex = adjustedIndex(index); - - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return false; - } - return true; - } - - @Override public boolean isInSnapshot(long index) { - return index <= snapshotIndex; - } - - @Override public Object getSnapshot() { - return snapshot; - } - - @Override public long getSnapshotIndex() { - return snapshotIndex; - } - - @Override public long getSnapshotTerm() { - return snapshotTerm; - } - - private int adjustedIndex(long index) { - if(snapshotIndex < 0){ - return (int) index; - } - return (int) (index - snapshotIndex); - } } - - private static class ReplicatedLogImplEntry implements ReplicatedLogEntry, - Serializable { - - private final long index; - private final long term; - private final Object payload; - - public ReplicatedLogImplEntry(long index, long term, Object payload) { - - this.index = index; - this.term = term; - this.payload = payload; - } - - @Override public Object getData() { - return payload; - } - - @Override public long getTerm() { - return term; - } - - @Override public long getIndex() { - return index; - } - - @Override public String toString() { - return "Entry{" + - "index=" + index + - ", term=" + term + - '}'; - } - } - - 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; } @@ -568,54 +776,6 @@ public abstract class RaftActor extends UntypedPersistentActor { } - private static class Snapshot implements Serializable { - private final Object state; - private final List unAppliedEntries; - private final long lastIndex; - private final long lastTerm; - private final long lastAppliedIndex; - private final long lastAppliedTerm; - - private Snapshot(Object state, - List unAppliedEntries, long lastIndex, - long lastTerm, long lastAppliedIndex, long lastAppliedTerm) { - this.state = state; - this.unAppliedEntries = unAppliedEntries; - this.lastIndex = lastIndex; - this.lastTerm = lastTerm; - this.lastAppliedIndex = lastAppliedIndex; - this.lastAppliedTerm = lastAppliedTerm; - } - - - public static Snapshot create(Object state, - List entries, long lastIndex, long lastTerm, - long lastAppliedIndex, long lastAppliedTerm) { - return new Snapshot(state, entries, lastIndex, lastTerm, - lastAppliedIndex, lastAppliedTerm); - } - - public Object getState() { - return state; - } - - public List getUnAppliedEntries() { - return unAppliedEntries; - } - - public long getLastTerm() { - return lastTerm; - } - - public long getLastAppliedIndex() { - return lastAppliedIndex; - } - - public long getLastAppliedTerm() { - return lastAppliedTerm; - } - } - private class ElectionTermImpl implements ElectionTerm { /** * Identifier of the actor whose election term information this is @@ -623,17 +783,20 @@ 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; } @Override public void update(long currentTerm, String votedFor) { - LOG.info("Set currentTerm={}, votedFor={}", currentTerm, votedFor); - + if(LOG.isDebugEnabled()) { + LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); + } this.currentTerm = currentTerm; this.votedFor = votedFor; } @@ -642,7 +805,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 { @@ -652,7 +815,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; @@ -670,4 +834,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; + } + }