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=0a979d24eef2a26d60cb6168c18d84815037cf18;hb=2a89ae48921724ef5a4ab42dcff6afc74c5b0a4a;hp=f38ef18973e8867b3825673cd07536b17a9293fd;hpb=5924885ac74b5fa0c729004a5b66b30654a55496;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 f38ef18973..3ec8cc5c58 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 @@ -10,29 +10,37 @@ 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 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 java.util.concurrent.TimeUnit; +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.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.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.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.protobuff.client.messages.Payload; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * RaftActor encapsulates a state machine that needs to be kept synchronized @@ -74,9 +82,11 @@ 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 @@ -88,77 +98,298 @@ 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 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(getSelf().path().toString()), - -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 = Stopwatch.createStarted(); + } + } - // 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(); + } + + @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(); + + initializeBehavior(); + } + } + } + + private void onRecoveredSnapshot(SnapshotOffer offer) { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: SnapshotOffer called..", persistenceId()); + } + + initRecoveryTimer(); + + Snapshot snapshot = (Snapshot) offer.snapshot(); - } else if (message instanceof ReplicatedLogEntry) { - replicatedLog.append((ReplicatedLogEntry) message); - } else if (message instanceof RecoveryCompleted) { - LOG.debug( - "Last index in log : " + replicatedLog.lastIndex()); - currentBehavior = switchBehavior(RaftState.Follower); + // 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 = Stopwatch.createStarted(); + + // Apply the snapshot to the actors state + applyRecoverySnapshot(snapshot.getState()); + + timer.stop(); + LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" + + replicatedLog.size(), persistenceId(), timer.toString(), + replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm()); + } + + private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex()); + } + + replicatedLog.append(logEntry); + } + + private void onRecoveredApplyLogEntries(ApplyLogEntries ale) { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Received ApplyLogEntries for recovery, applying to state: {} to {}", + persistenceId(), 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(); } } - @Override public void onReceiveCommand(Object message) { - if (message instanceof ApplyState) { + 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.getSnapshotIndex(), + replicatedLog.getSnapshotTerm(), replicatedLog.size()); + + initializeBehavior(); + } + + protected void initializeBehavior(){ + changeCurrentBehavior(new Follower(context)); + } + + protected void changeCurrentBehavior(RaftActorBehavior newBehavior){ + RaftActorBehavior oldBehavior = currentBehavior; + currentBehavior = newBehavior; + handleBehaviorChange(oldBehavior, currentBehavior); + } + @Override public void handleCommand(Object message) { + if (message instanceof ApplyState){ ApplyState applyState = (ApplyState) message; - LOG.debug("Applying state for log index {}", - applyState.getReplicatedLogEntry().getIndex()); + 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 {}", + persistenceId(), 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={}", persistenceId(), 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:{}", persistenceId(), snapshot.getLastAppliedIndex(), + snapshot.getLastAppliedTerm() + ); + } + + applySnapshot(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", persistenceId()); + + 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) { - // TODO: Handle failure in saving the snapshot + SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; + + LOG.error("{}: SaveSnapshotFailure received for snapshot Cause:", + persistenceId(), saveSnapshotFailure.cause()); + + context.getReplicatedLog().snapshotRollback(); + + LOG.info("{}: Replicated Log rollbacked. Snapshot will be attempted in the next cycle." + + "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(), + context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm(), + context.getReplicatedLog().size()); + + } else if (message instanceof CaptureSnapshot) { + LOG.info("{}: CaptureSnapshot received by actor", persistenceId()); + + if(captureSnapshot == null) { + captureSnapshot = (CaptureSnapshot)message; + createSnapshot(); + } + + } else if (message instanceof CaptureSnapshotReply){ + handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot()); + } else { - RaftState state = - currentBehavior.handleMessage(getSender(), message); - currentBehavior = switchBehavior(state); + RaftActorBehavior oldBehavior = currentBehavior; + currentBehavior = currentBehavior.handleMessage(getSender(), message); + + handleBehaviorChange(oldBehavior, currentBehavior); } } + private void handleBehaviorChange(RaftActorBehavior oldBehavior, RaftActorBehavior currentBehavior) { + if (oldBehavior != currentBehavior){ + onStateChanged(); + } + + String oldBehaviorLeaderId = oldBehavior == null? null : oldBehavior.getLeaderId(); + String oldBehaviorState = oldBehavior == null? null : oldBehavior.state().name(); + // it can happen that the state has not changed but the leader has changed. + onLeaderChanged(oldBehaviorLeaderId, currentBehavior.getLeaderId()); + + if (getRoleChangeNotifier().isPresent() && + (oldBehavior == null || (oldBehavior.state() != currentBehavior.state()))) { + getRoleChangeNotifier().get().tell( + new RoleChanged(getId(), oldBehaviorState , currentBehavior.state().name()), + getSelf()); + } + } /** * When a derived RaftActor needs to persist something it must call @@ -168,16 +399,54 @@ public abstract class RaftActor extends UntypedPersistentActor { * @param identifier * @param data */ - protected void persistData(ActorRef clientActor, String identifier, - Object data) { - LOG.debug("Persist data " + identifier); + 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 {}", persistenceId(), replicatedLogEntry); + } + + final RaftActorContext raftContext = getRaftActorContext(); + replicatedLog - .appendAndPersist(clientActor, identifier, replicatedLogEntry); - } + .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 + applyState(clientActor, identifier, data); + + // Send a ApplyLogEntries message so that we write the fact that we applied + // the state to durable storage + self().tell(new ApplyLogEntries((int) replicatedLogEntry.getIndex()), self()); + + // Check if the "real" snapshot capture has been initiated. If no then do the fake snapshot + if(!context.isSnapshotCaptureInitiated()){ + raftContext.getReplicatedLog().snapshotPreCommit(raftContext.getLastApplied(), + raftContext.getTermInformation().getCurrentTerm()); + raftContext.getReplicatedLog().snapshotCommit(); + } else { + LOG.debug("{}: Skipping fake snapshotting for {} because real snapshotting is in progress", + persistenceId(), getId()); + } + } else if (clientActor != null) { + // Send message for replication + currentBehavior.handleMessage(getSelf(), + new Replicate(clientActor, identifier, + replicatedLogEntry) + ); + } + + } + }); } protected String getId() { return context.getId(); @@ -200,15 +469,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) { + protected ActorSelection getLeader(){ + 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); } /** @@ -231,6 +556,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 snapshotBytes A snapshot of the state of the actor + */ + protected abstract void applyRecoverySnapshot(byte[] snapshotBytes); + + /** + * 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. @@ -240,155 +597,162 @@ 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 + * @param snapshotBytes A snapshot of the state of the actor */ - protected abstract void applySnapshot(Object snapshot); + protected abstract void applySnapshot(byte[] snapshotBytes); - 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; + private String getLeaderAddress(){ + if(isLeader()){ + return getSelf().path().toString(); + } + String leaderId = currentBehavior.getLeaderId(); + if (leaderId == null) { + return null; + } + String peerAddress = context.getPeerAddress(leaderId); + if(LOG.isDebugEnabled()) { + LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}", + persistenceId(), leaderId, peerAddress); + } - public ReplicatedLogImpl(Snapshot snapshot) { - this.snapshot = snapshot.getState(); - this.snapshotIndex = snapshot.getLastAppliedIndex(); - this.snapshotTerm = snapshot.getLastAppliedTerm(); + return peerAddress; + } - this.journal = new ArrayList<>(snapshot.getUnAppliedEntries()); - } + private void handleCaptureSnapshotReply(byte[] snapshotBytes) { + LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length); - public ReplicatedLogImpl() { - this.snapshot = null; - this.journal = new ArrayList<>(); - } + // create a snapshot object from the state provided and save it + // when snapshot is saved async, SaveSnapshotSuccess is raised. - @Override public ReplicatedLogEntry get(long index) { - int adjustedIndex = adjustedIndex(index); + Snapshot sn = Snapshot.create(snapshotBytes, + context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), + captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), + captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return null; - } + persistence().saveSnapshot(sn); - return journal.get(adjustedIndex); - } + LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage()); - @Override public ReplicatedLogEntry last() { - if (journal.size() == 0) { - return null; - } - return get(journal.size() - 1); + long dataThreshold = Runtime.getRuntime().totalMemory() * + getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100; + if (context.getReplicatedLog().dataSize() > dataThreshold) { + // if memory is less, clear the log based on lastApplied. + // this could/should only happen if one of the followers is down + // as normally we keep removing from the log when its replicated to all. + context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); + + } else { + // clear the log based on replicatedToAllIndex + context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(), + captureSnapshot.getReplicatedToAllTerm()); } + getCurrentBehavior().setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex()); - @Override public long lastIndex() { - if (journal.size() == 0) { - return -1; - } + LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + + "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(), + captureSnapshot.getLastAppliedTerm()); - return last().getIndex(); + if (isLeader() && captureSnapshot.isInstallSnapshotInitiated()) { + // this would be call straight to the leader and won't initiate in serialization + currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot( + ByteString.copyFrom(snapshotBytes))); } - @Override public long lastTerm() { - if (journal.size() == 0) { - return -1; - } + captureSnapshot = null; + context.setSnapshotCaptureInitiated(false); + } - return last().getTerm(); - } + protected boolean hasFollowers(){ + return getRaftActorContext().getPeerAddresses().keySet().size() > 0; + } + private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { - @Override public void removeFrom(long index) { - int adjustedIndex = adjustedIndex(index); + private static final int DATA_SIZE_DIVIDER = 5; + private long dataSizeSinceLastSnapshot = 0; - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return; - } - for (int i = adjustedIndex; - i < journal.size(); i++) { - deleteMessage(i); - journal.remove(i); - } + public ReplicatedLogImpl(Snapshot snapshot) { + super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), + snapshot.getUnAppliedEntries()); } - @Override public void append( - final ReplicatedLogEntry replicatedLogEntry) { - journal.add(replicatedLogEntry); + public ReplicatedLogImpl() { + super(); } - @Override public List getFrom(long index) { - int adjustedIndex = adjustedIndex(index); + @Override public void removeFromAndPersist(long logEntryIndex) { + int adjustedIndex = adjustedIndex(logEntryIndex); - 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)); + if (adjustedIndex < 0) { + return; } - return entries; + + // FIXME: Maybe this should be done after the command is saved + journal.subList(adjustedIndex , journal.size()).clear(); + + 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 appendAndPersist( final ReplicatedLogEntry replicatedLogEntry) { - appendAndPersist(null, null, replicatedLogEntry); + appendAndPersist(replicatedLogEntry, null); } - public void appendAndPersist(final ActorRef clientActor, - final String identifier, - final ReplicatedLogEntry replicatedLogEntry) { - context.getLogger().debug( - "Append log entry and persist " + replicatedLogEntry.getIndex()); + public void appendAndPersist( + final ReplicatedLogEntry replicatedLogEntry, + final Procedure callback) { + + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Append log entry and persist {} ", persistenceId(), 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); @@ -397,150 +761,191 @@ 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()); + int logEntrySize = replicatedLogEntry.size(); + + dataSize += logEntrySize; + long dataSizeForCheck = dataSize; + + dataSizeSinceLastSnapshot += logEntrySize; + long journalSize = lastIndex() + 1; + + if(!hasFollowers()) { + // When we do not have followers we do not maintain an in-memory log + // due to this the journalSize will never become anything close to the + // snapshot batch count. In fact will mostly be 1. + // Similarly since the journal's dataSize depends on the entries in the + // journal the journal's dataSize will never reach a value close to the + // memory threshold. + // By maintaining the dataSize outside the journal we are tracking essentially + // what we have written to the disk however since we no longer are in + // need of doing a snapshot just for the sake of freeing up memory we adjust + // the real size of data by the DATA_SIZE_DIVIDER so that we do not snapshot as often + // as if we were maintaining a real snapshot + dataSizeForCheck = dataSizeSinceLastSnapshot / DATA_SIZE_DIVIDER; + } + + long dataThreshold = Runtime.getRuntime().totalMemory() * + getRaftActorContext().getConfigParams().getSnapshotDataThresholdPercentage() / 100; + + // when a snaphsot is being taken, captureSnapshot != null + if (!context.isSnapshotCaptureInitiated() && + ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 || + dataSizeForCheck > dataThreshold)) { + + dataSizeSinceLastSnapshot = 0; + + LOG.info("{}: Initiating Snapshot Capture, journalSize = {}, dataSizeForCheck = {}," + + " dataThreshold = {}", persistenceId(), journalSize, dataSizeForCheck, dataThreshold); + long lastAppliedIndex = -1; long lastAppliedTerm = -1; - if (lastAppliedEntry != null) { + + ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied()); + if (!hasFollowers()) { + lastAppliedIndex = replicatedLogEntry.getIndex(); + lastAppliedTerm = replicatedLogEntry.getTerm(); + } else 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: {}", persistenceId(), journal.size()); + LOG.debug("{}: Snapshot Capture lastApplied:{} ", + persistenceId(), context.getLastApplied()); + LOG.debug("{}: Snapshot Capture lastAppliedIndex:{}", persistenceId(), + lastAppliedIndex); + LOG.debug("{}: Snapshot Capture lastAppliedTerm:{}", persistenceId(), + lastAppliedTerm); + } + + // send a CaptureSnapshot to self to make the expensive operation async. + long replicatedToAllIndex = getCurrentBehavior().getReplicatedToAllIndex(); + ReplicatedLogEntry replicatedToAllEntry = context.getReplicatedLog().get(replicatedToAllIndex); + getSelf().tell(new CaptureSnapshot(lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm, + (replicatedToAllEntry != null ? replicatedToAllEntry.getIndex() : -1), + (replicatedToAllEntry != null ? replicatedToAllEntry.getTerm() : -1)), + null); + context.setSnapshotCaptureInitiated(true); } - // Send message for replication - if (clientActor != null) { - currentBehavior.handleMessage(getSelf(), - new Replicate(clientActor, identifier, - replicatedLogEntry) - ); + if (callback != null){ + callback.apply(replicatedLogEntry); } } } ); } - @Override public long size() { - return journal.size() + snapshotIndex; - } + } - @Override public boolean isPresent(long index) { - int adjustedIndex = adjustedIndex(index); + static class DeleteEntries implements Serializable { + private static final long serialVersionUID = 1L; + private final int fromIndex; - if (adjustedIndex < 0 || adjustedIndex >= journal.size()) { - return false; - } - return true; + public DeleteEntries(int fromIndex) { + this.fromIndex = fromIndex; } - @Override public boolean isInSnapshot(long index) { - return index <= snapshotIndex; + public int getFromIndex() { + return fromIndex; } + } - @Override public Object getSnapshot() { - return snapshot; - } - @Override public long getSnapshotIndex() { - return snapshotIndex; - } + private class ElectionTermImpl implements ElectionTerm { + /** + * Identifier of the actor whose election term information this is + */ + private long currentTerm = 0; + private String votedFor = null; - @Override public long getSnapshotTerm() { - return snapshotTerm; + @Override + public long getCurrentTerm() { + return currentTerm; } - private int adjustedIndex(long index) { - if(snapshotIndex < 0){ - return (int) index; - } - return (int) (index - snapshotIndex); + @Override + public String getVotedFor() { + return votedFor; } - } - - 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 void update(long currentTerm, String votedFor) { + if(LOG.isDebugEnabled()) { + LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor); + } + this.currentTerm = currentTerm; + this.votedFor = votedFor; } - @Override public Object getData() { - return payload; - } + @Override + public void updateAndPersist(long currentTerm, String votedFor){ + update(currentTerm, votedFor); + // FIXME : Maybe first persist then update the state + persistence().persist(new UpdateElectionTerm(this.currentTerm, this.votedFor), new Procedure(){ - @Override public long getTerm() { - return term; - } + @Override public void apply(UpdateElectionTerm param) + throws Exception { - @Override public long getIndex() { - return index; + } + }); } } + static class UpdateElectionTerm implements Serializable { + private static final long serialVersionUID = 1L; + private final long currentTerm; + private final String votedFor; - 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 UpdateElectionTerm(long currentTerm, String votedFor) { + this.currentTerm = currentTerm; + this.votedFor = votedFor; } - - 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 long getCurrentTerm() { + return currentTerm; } - public Object getState() { - return state; + public String getVotedFor() { + return votedFor; } + } - public List getUnAppliedEntries() { - return unAppliedEntries; - } + protected class NonPersistentRaftDataProvider extends NonPersistentDataProvider { - public long getLastTerm() { - return lastTerm; - } + public NonPersistentRaftDataProvider(){ - public long getLastAppliedIndex() { - return lastAppliedIndex; } - public long getLastAppliedTerm() { - return lastAppliedTerm; + /** + * 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; + } }