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=3ec8cc5c5817d92825ba882101d21c5b437863cd;hp=f02b52beb9aa2c6aa14bbd5cc71dae725690f817;hb=2a89ae48921724ef5a4ab42dcff6afc74c5b0a4a;hpb=2b0c99463883b10d5eacdec901d7543d5815a54f 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 f02b52beb9..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,8 +10,6 @@ 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; @@ -22,29 +20,27 @@ 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.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.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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * RaftActor encapsulates a state machine that needs to be kept synchronized @@ -87,8 +83,10 @@ import java.util.Map; * */ public abstract class RaftActor extends AbstractUntypedPersistentActor { - protected final LoggingAdapter LOG = - Logging.getLogger(getContext().system(), this); + + 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 @@ -109,8 +107,6 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { private CaptureSnapshot captureSnapshot = null; - private volatile boolean hasSnapshotCaptureInitiated = false; - private Stopwatch recoveryTimer; private int currentRecoveryBatchCount; @@ -131,8 +127,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { private void initRecoveryTimer() { if(recoveryTimer == null) { - recoveryTimer = new Stopwatch(); - recoveryTimer.start(); + recoveryTimer = Stopwatch.createStarted(); } } @@ -171,15 +166,15 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(), scala.Long.MaxValue())); onRecoveryComplete(); - currentBehavior = new Follower(context); - onStateChanged(); + + initializeBehavior(); } } } private void onRecoveredSnapshot(SnapshotOffer offer) { if(LOG.isDebugEnabled()) { - LOG.debug("SnapshotOffer called.."); + LOG.debug("{}: SnapshotOffer called..", persistenceId()); } initRecoveryTimer(); @@ -195,21 +190,20 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { context.setLastApplied(snapshot.getLastAppliedIndex()); context.setCommitIndex(snapshot.getLastAppliedIndex()); - Stopwatch timer = new Stopwatch(); - timer.start(); + Stopwatch timer = Stopwatch.createStarted(); // Apply the snapshot to the actors state - applyRecoverySnapshot(ByteString.copyFrom(snapshot.getState())); + applyRecoverySnapshot(snapshot.getState()); timer.stop(); LOG.info("Recovery snapshot applied for {} in {}: snapshotIndex={}, snapshotTerm={}, journal-size=" + replicatedLog.size(), persistenceId(), timer.toString(), - replicatedLog.snapshotIndex, replicatedLog.snapshotTerm); + replicatedLog.getSnapshotIndex(), replicatedLog.getSnapshotTerm()); } private void onRecoveredJournalLogEntry(ReplicatedLogEntry logEntry) { if(LOG.isDebugEnabled()) { - LOG.debug("Received ReplicatedLogEntry for recovery: {}", logEntry.getIndex()); + LOG.debug("{}: Received ReplicatedLogEntry for recovery: {}", persistenceId(), logEntry.getIndex()); } replicatedLog.append(logEntry); @@ -217,8 +211,8 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { private void onRecoveredApplyLogEntries(ApplyLogEntries ale) { if(LOG.isDebugEnabled()) { - LOG.debug("Received ApplyLogEntries for recovery, applying to state: {} to {}", - context.getLastApplied() + 1, ale.getToIndex()); + 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++) { @@ -268,20 +262,35 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { "Persistence Id = " + persistenceId() + " Last index in log={}, snapshotIndex={}, snapshotTerm={}, " + "journal-size={}", - replicatedLog.lastIndex(), replicatedLog.snapshotIndex, - replicatedLog.snapshotTerm, replicatedLog.size()); + replicatedLog.lastIndex(), replicatedLog.getSnapshotIndex(), + replicatedLog.getSnapshotTerm(), replicatedLog.size()); - currentBehavior = new Follower(context); - onStateChanged(); + 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; + long elapsedTime = (System.nanoTime() - applyState.getStartTime()); + if(elapsedTime >= APPLY_STATE_DELAY_THRESHOLD_IN_NANOS){ + LOG.warn("ApplyState took more time than expected. Elapsed Time = {} ms ApplyState = {}", + TimeUnit.NANOSECONDS.toMillis(elapsedTime), applyState); + } + if(LOG.isDebugEnabled()) { - LOG.debug("Applying state for log index {} data {}", - applyState.getReplicatedLogEntry().getIndex(), + LOG.debug("{}: Applying state for log index {} data {}", + persistenceId(), applyState.getReplicatedLogEntry().getIndex(), applyState.getReplicatedLogEntry().getData()); } @@ -291,7 +300,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } else if (message instanceof ApplyLogEntries){ ApplyLogEntries ale = (ApplyLogEntries) message; if(LOG.isDebugEnabled()) { - LOG.debug("Persisting ApplyLogEntries with index={}", ale.getToIndex()); + LOG.debug("{}: Persisting ApplyLogEntries with index={}", persistenceId(), ale.getToIndex()); } persistence().persist(new ApplyLogEntries(ale.getToIndex()), new Procedure() { @Override @@ -303,12 +312,13 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { Snapshot snapshot = ((ApplySnapshot) message).getSnapshot(); if(LOG.isDebugEnabled()) { - LOG.debug("ApplySnapshot called on Follower Actor " + - "snapshotIndex:{}, snapshotTerm:{}", snapshot.getLastAppliedIndex(), + LOG.debug("{}: ApplySnapshot called on Follower Actor " + + "snapshotIndex:{}, snapshotTerm:{}", persistenceId(), snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm() ); } - applySnapshot(ByteString.copyFrom(snapshot.getState())); + + applySnapshot(snapshot.getState()); //clears the followers log, sets the snapshot index to ensure adjusted-index works replicatedLog = new ReplicatedLogImpl(snapshot); @@ -323,7 +333,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } else if (message instanceof SaveSnapshotSuccess) { SaveSnapshotSuccess success = (SaveSnapshotSuccess) message; - LOG.info("SaveSnapshotSuccess received for snapshot"); + LOG.info("{}: SaveSnapshotSuccess received for snapshot", persistenceId()); long sequenceNumber = success.metadata().sequenceNr(); @@ -332,76 +342,55 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } else if (message instanceof SaveSnapshotFailure) { SaveSnapshotFailure saveSnapshotFailure = (SaveSnapshotFailure) message; - LOG.info("saveSnapshotFailure.metadata():{}", saveSnapshotFailure.metadata().toString()); - LOG.error(saveSnapshotFailure.cause(), "SaveSnapshotFailure received for snapshot Cause:"); + 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:{}", + 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 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; - captureSnapshot = cs; - createSnapshot(); + LOG.info("{}: CaptureSnapshot received by actor", persistenceId()); - } else if (message instanceof CaptureSnapshotReply){ - LOG.info("CaptureSnapshotReply received by actor"); - CaptureSnapshotReply csr = (CaptureSnapshotReply) message; + if(captureSnapshot == null) { + captureSnapshot = (CaptureSnapshot)message; + createSnapshot(); + } - ByteString stateInBytes = csr.getSnapshot(); - LOG.info("CaptureSnapshotReply stateInBytes size:{}", stateInBytes.size()); - handleCaptureSnapshotReply(stateInBytes); + } else if (message instanceof CaptureSnapshotReply){ + handleCaptureSnapshotReply(((CaptureSnapshotReply) message).getSnapshot()); } 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); - if(oldBehavior != currentBehavior){ - onStateChanged(); - } - - onLeaderChanged(oldBehavior.getLeaderId(), currentBehavior.getLeaderId()); + handleBehaviorChange(oldBehavior, currentBehavior); } } - public java.util.Set getPeers() { + 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(); - return context.getPeerAddresses().keySet(); - } + // it can happen that the state has not changed but the leader has changed. + onLeaderChanged(oldBehaviorLeaderId, 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 == 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 * persistData. @@ -410,20 +399,54 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { * @param identifier * @param data */ - protected void persistData(ActorRef clientActor, String identifier, - Payload data) { + protected void persistData(final ActorRef clientActor, final String identifier, + final Payload data) { ReplicatedLogEntry replicatedLogEntry = new ReplicatedLogImplEntry( context.getReplicatedLog().lastIndex() + 1, context.getTermInformation().getCurrentTerm(), data); if(LOG.isDebugEnabled()) { - LOG.debug("Persist data {}", replicatedLogEntry); + LOG.debug("{}: Persist data {}", persistenceId(), replicatedLogEntry); } + 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(); @@ -550,9 +573,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { /** * This method is called during recovery to reconstruct the state of the actor. * - * @param snapshot A snapshot of the state of the actor + * @param snapshotBytes A snapshot of the state of the actor */ - protected abstract void applyRecoverySnapshot(ByteString snapshot); + protected abstract void applyRecoverySnapshot(byte[] snapshotBytes); /** * This method is called during recovery at the end of a batch to apply the current batched @@ -581,9 +604,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { * 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(ByteString snapshot); + protected abstract void applySnapshot(byte[] snapshotBytes); /** * This method will be called by the RaftActor when the state of the @@ -594,6 +617,12 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { 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) { @@ -617,48 +646,67 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } String peerAddress = context.getPeerAddress(leaderId); if(LOG.isDebugEnabled()) { - LOG.debug("getLeaderAddress leaderId = {} peerAddress = {}", - leaderId, peerAddress); + LOG.debug("{}: getLeaderAddress leaderId = {} peerAddress = {}", + persistenceId(), leaderId, peerAddress); } return peerAddress; } - private void handleCaptureSnapshotReply(ByteString stateInBytes) { + private void handleCaptureSnapshotReply(byte[] snapshotBytes) { + LOG.info("{}: CaptureSnapshotReply received by actor: snapshot size {}", persistenceId(), snapshotBytes.length); + // create a snapshot object from the state provided and save it // when snapshot is saved async, SaveSnapshotSuccess is raised. - Snapshot sn = Snapshot.create(stateInBytes.toByteArray(), + Snapshot sn = Snapshot.create(snapshotBytes, context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); persistence().saveSnapshot(sn); - LOG.info("Persisting of snapshot done:{}", sn.getLogMessage()); + LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), 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, + 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()); - 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()); - LOG.info("Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + - "and term:{}", captureSnapshot.getLastAppliedIndex(), + LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + + "and term:{}", persistenceId(), 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)); + currentBehavior.handleMessage(getSelf(), new SendInstallSnapshot( + ByteString.copyFrom(snapshotBytes))); } captureSnapshot = null; - hasSnapshotCaptureInitiated = false; + context.setSnapshotCaptureInitiated(false); + } + + protected boolean hasFollowers(){ + return getRaftActorContext().getPeerAddresses().keySet().size() > 0; } private class ReplicatedLogImpl extends AbstractReplicatedLogImpl { + private static final int DATA_SIZE_DIVIDER = 5; + private long dataSizeSinceLastSnapshot = 0; + public ReplicatedLogImpl(Snapshot snapshot) { super(snapshot.getLastAppliedIndex(), snapshot.getLastAppliedTerm(), snapshot.getUnAppliedEntries()); @@ -678,26 +726,31 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { // FIXME: Maybe this should be done after the command is saved journal.subList(adjustedIndex , journal.size()).clear(); - persistence().persist(new DeleteEntries(adjustedIndex), new Procedure(){ + persistence().persist(new DeleteEntries(adjustedIndex), new Procedure() { - @Override public void apply(DeleteEntries param) - throws Exception { + @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) { + public void appendAndPersist( + final ReplicatedLogEntry replicatedLogEntry, + final Procedure callback) { if(LOG.isDebugEnabled()) { - LOG.debug("Append log entry and persist {} ", replicatedLogEntry); + 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 @@ -712,40 +765,75 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { new Procedure() { @Override public void apply(ReplicatedLogEntry evt) throws Exception { + 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 (hasSnapshotCaptureInitiated == false && - journal.size() % context.getConfigParams().getSnapshotBatchCount() == 0) { + if (!context.isSnapshotCaptureInitiated() && + ( journalSize % context.getConfigParams().getSnapshotBatchCount() == 0 || + dataSizeForCheck > dataThreshold)) { + + dataSizeSinceLastSnapshot = 0; + + LOG.info("{}: Initiating Snapshot Capture, journalSize = {}, dataSizeForCheck = {}," + + " dataThreshold = {}", persistenceId(), journalSize, dataSizeForCheck, dataThreshold); - LOG.info("Initiating Snapshot Capture.."); long lastAppliedIndex = -1; long lastAppliedTerm = -1; ReplicatedLogEntry lastAppliedEntry = get(context.getLastApplied()); - if (lastAppliedEntry != null) { + if (!hasFollowers()) { + lastAppliedIndex = replicatedLogEntry.getIndex(); + lastAppliedTerm = replicatedLogEntry.getTerm(); + } else if (lastAppliedEntry != null) { lastAppliedIndex = lastAppliedEntry.getIndex(); lastAppliedTerm = lastAppliedEntry.getTerm(); } if(LOG.isDebugEnabled()) { - LOG.debug("Snapshot Capture logSize: {}", journal.size()); - LOG.debug("Snapshot Capture lastApplied:{} ", - context.getLastApplied()); - LOG.debug("Snapshot Capture lastAppliedIndex:{}", lastAppliedIndex); - LOG.debug("Snapshot Capture lastAppliedTerm:{}", lastAppliedTerm); + 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. - getSelf().tell(new CaptureSnapshot( - lastIndex(), lastTerm(), lastAppliedIndex, lastAppliedTerm), + 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); - hasSnapshotCaptureInitiated = true; + context.setSnapshotCaptureInitiated(true); } - // Send message for replication - if (clientActor != null) { - currentBehavior.handleMessage(getSelf(), - new Replicate(clientActor, identifier, - replicatedLogEntry) - ); + if (callback != null){ + callback.apply(replicatedLogEntry); } } } @@ -755,9 +843,9 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } static class DeleteEntries implements Serializable { + private static final long serialVersionUID = 1L; private final int fromIndex; - public DeleteEntries(int fromIndex) { this.fromIndex = fromIndex; } @@ -787,7 +875,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { @Override public void update(long currentTerm, String votedFor) { if(LOG.isDebugEnabled()) { - LOG.debug("Set currentTerm={}, votedFor={}", currentTerm, votedFor); + LOG.debug("{}: Set currentTerm={}, votedFor={}", persistenceId(), currentTerm, votedFor); } this.currentTerm = currentTerm; this.votedFor = votedFor; @@ -808,6 +896,7 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { } static class UpdateElectionTerm implements Serializable { + private static final long serialVersionUID = 1L; private final long currentTerm; private final String votedFor; @@ -858,4 +947,5 @@ public abstract class RaftActor extends AbstractUntypedPersistentActor { protected RaftActorBehavior getCurrentBehavior() { return currentBehavior; } + }