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%2FSnapshotManager.java;h=4c3fb109add982c9f2d5590d524d20dfbf7ea7b7;hb=4c0cc4831e7ecc2d5f745a42ba7a390361432fd2;hp=cb38e82ac3c7f5bb42cee275e3ed1f24a423ee0b;hpb=9fe7a995204bcfed3ee6b644922b8fe440fe5f5c;p=controller.git diff --git a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SnapshotManager.java b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SnapshotManager.java index cb38e82ac3..4c3fb109ad 100644 --- a/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SnapshotManager.java +++ b/opendaylight/md-sal/sal-akka-raft/src/main/java/org/opendaylight/controller/cluster/raft/SnapshotManager.java @@ -10,8 +10,8 @@ package org.opendaylight.controller.cluster.raft; import akka.japi.Procedure; import akka.persistence.SnapshotSelectionCriteria; -import com.google.protobuf.ByteString; -import org.opendaylight.controller.cluster.DataPersistenceProvider; +import com.google.common.annotations.VisibleForTesting; +import java.util.List; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot; import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot; import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; @@ -19,9 +19,7 @@ import org.slf4j.Logger; public class SnapshotManager implements SnapshotState { - private final SnapshotState IDLE = new Idle(); - private final SnapshotState CAPTURING = new Capturing(); private final SnapshotState PERSISTING = new Persisting(); private final SnapshotState CREATING = new Creating(); @@ -35,6 +33,12 @@ public class SnapshotManager implements SnapshotState { private SnapshotState currentState = IDLE; private CaptureSnapshot captureSnapshot; + private long lastSequenceNumber = -1; + + private Procedure createSnapshotProcedure; + + private Snapshot applySnapshot; + private Procedure applySnapshotProcedure; public SnapshotManager(RaftActorContext context, Logger logger) { this.context = context; @@ -47,28 +51,28 @@ public class SnapshotManager implements SnapshotState { } @Override - public void captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { - currentState.captureToInstall(lastLogEntry, replicatedToAllIndex); + public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { + return currentState.captureToInstall(lastLogEntry, replicatedToAllIndex, targetFollower); } @Override - public void capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { - currentState.capture(lastLogEntry, replicatedToAllIndex); + public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { + return currentState.capture(lastLogEntry, replicatedToAllIndex); } @Override - public void create(Procedure callback) { - currentState.create(callback); + public void apply(Snapshot snapshot) { + currentState.apply(snapshot); } @Override - public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes, RaftActorBehavior currentBehavior) { - currentState.persist(persistenceProvider, snapshotBytes, currentBehavior); + public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { + currentState.persist(snapshotBytes, currentBehavior, totalMemory); } @Override - public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) { - currentState.commit(persistenceProvider, sequenceNumber); + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { + currentState.commit(sequenceNumber, currentBehavior); } @Override @@ -77,8 +81,25 @@ public class SnapshotManager implements SnapshotState { } @Override - public long trimLog(long desiredTrimIndex) { - return currentState.trimLog(desiredTrimIndex); + public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { + return currentState.trimLog(desiredTrimIndex, currentBehavior); + } + + public void setCreateSnapshotCallable(Procedure createSnapshotProcedure) { + this.createSnapshotProcedure = createSnapshotProcedure; + } + + public void setApplySnapshotProcedure(Procedure applySnapshotProcedure) { + this.applySnapshotProcedure = applySnapshotProcedure; + } + + public long getLastSequenceNumber() { + return lastSequenceNumber; + } + + @VisibleForTesting + public CaptureSnapshot getCaptureSnapshot() { + return captureSnapshot; } private boolean hasFollowers(){ @@ -93,31 +114,33 @@ public class SnapshotManager implements SnapshotState { @Override public boolean isCapturing() { - return false; + return true; } @Override - public void capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { + public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { LOG.debug("capture should not be called in state {}", this); + return false; } @Override - public void captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { + public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { LOG.debug("captureToInstall should not be called in state {}", this); + return false; } @Override - public void create(Procedure callback) { - LOG.debug("create should not be called in state {}", this); + public void apply(Snapshot snapshot) { + LOG.debug("apply should not be called in state {}", this); } @Override - public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes, RaftActorBehavior currentBehavior) { + public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { LOG.debug("persist should not be called in state {}", this); } @Override - public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) { + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { LOG.debug("commit should not be called in state {}", this); } @@ -127,17 +150,22 @@ public class SnapshotManager implements SnapshotState { } @Override - public long trimLog(long desiredTrimIndex) { + public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { LOG.debug("trimLog should not be called in state {}", this); return -1; } - protected long doTrimLog(long desiredTrimIndex){ + protected long doTrimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior){ // we would want to keep the lastApplied as its used while capturing snapshots long lastApplied = context.getLastApplied(); long tempMin = Math.min(desiredTrimIndex, (lastApplied > -1 ? lastApplied - 1 : -1)); - if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) { + if(LOG.isTraceEnabled()) { + LOG.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, tempMin: {}", + persistenceId(), desiredTrimIndex, lastApplied, tempMin); + } + + if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) { LOG.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin, context.getTermInformation().getCurrentTerm()); @@ -146,15 +174,26 @@ public class SnapshotManager implements SnapshotState { context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm()); context.getReplicatedLog().snapshotCommit(); return tempMin; + } else if(tempMin > currentBehavior.getReplicatedToAllIndex()) { + // It's possible a follower was lagging and an install snapshot advanced its match index past + // the current replicatedToAllIndex. Since the follower is now caught up we should advance the + // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely + // due to a previous snapshot triggered by the memory threshold exceeded, in that case we + // trim the log to the last applied index even if previous entries weren't replicated to all followers. + currentBehavior.setReplicatedToAllIndex(tempMin); } - return -1; } } private class Idle extends AbstractSnapshotState { - private void capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, boolean toInstall) { + @Override + public boolean isCapturing() { + return false; + } + + private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { TermInformationReader lastAppliedTermInfoReader = lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(), lastLogEntry, hasFollowers()); @@ -169,95 +208,130 @@ public class SnapshotManager implements SnapshotState { long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm(); // send a CaptureSnapshot to self to make the expensive operation async. - captureSnapshot = new CaptureSnapshot(lastLogEntry.getIndex(), - lastLogEntry.getTerm(), lastAppliedIndex, lastAppliedTerm, - newReplicatedToAllIndex, newReplicatedToAllTerm, toInstall); - SnapshotManager.this.currentState = CAPTURING; + List unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1); - LOG.info("{}: Initiating snapshot capture {}: {}", persistenceId(), toInstall ? "to install" : "", - captureSnapshot); + long lastLogEntryIndex = lastAppliedIndex; + long lastLogEntryTerm = lastAppliedTerm; + if(lastLogEntry != null) { + lastLogEntryIndex = lastLogEntry.getIndex(); + lastLogEntryTerm = lastLogEntry.getTerm(); + } else { + LOG.warn("Capturing Snapshot : lastLogEntry is null. Using lastAppliedIndex {} and lastAppliedTerm {} instead.", + lastAppliedIndex, lastAppliedTerm); + } - context.getActor().tell(captureSnapshot, context.getActor()); - } + captureSnapshot = new CaptureSnapshot(lastLogEntryIndex, + lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm, + newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, targetFollower != null); - @Override - public void capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { - capture(lastLogEntry, replicatedToAllIndex, false); + if(captureSnapshot.isInstallSnapshotInitiated()) { + LOG.info("{}: Initiating snapshot capture {} to install on {}", + persistenceId(), captureSnapshot, targetFollower); + } else { + LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot); + } + + lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber(); + + LOG.debug("lastSequenceNumber prior to capture: {}", lastSequenceNumber); + + SnapshotManager.this.currentState = CREATING; + + try { + createSnapshotProcedure.apply(null); + } catch (Exception e) { + SnapshotManager.this.currentState = IDLE; + LOG.error("Error creating snapshot", e); + return false; + } + + return true; } @Override - public void captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { - capture(lastLogEntry, replicatedToAllIndex, true); + public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { + return capture(lastLogEntry, replicatedToAllIndex, null); } @Override - public String toString() { - return "Idle"; + public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { + return capture(lastLogEntry, replicatedToAllIndex, targetFollower); } @Override - public long trimLog(long desiredTrimIndex) { - return doTrimLog(desiredTrimIndex); - } - } + public void apply(Snapshot snapshot) { + applySnapshot = snapshot; - private class Capturing extends AbstractSnapshotState { + lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber(); - @Override - public boolean isCapturing() { - return true; - } + LOG.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber); - @Override - public void create(Procedure callback) { - try { - callback.apply(null); - SnapshotManager.this.currentState = CREATING; - } catch (Exception e) { - LOG.error("Unexpected error occurred", e); - } + context.getPersistenceProvider().saveSnapshot(snapshot); + + SnapshotManager.this.currentState = PERSISTING; } @Override public String toString() { - return "Capturing"; + return "Idle"; } + @Override + public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { + return doTrimLog(desiredTrimIndex, currentBehavior); + } } private class Creating extends AbstractSnapshotState { @Override - public boolean isCapturing() { - return true; - } - - @Override - public void persist(DataPersistenceProvider persistenceProvider, byte[] snapshotBytes, - RaftActorBehavior currentBehavior) { + public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { // create a snapshot object from the state provided and save it // when snapshot is saved async, SaveSnapshotSuccess is raised. - Snapshot sn = Snapshot.create(snapshotBytes, - context.getReplicatedLog().getFrom(captureSnapshot.getLastAppliedIndex() + 1), + Snapshot snapshot = Snapshot.create(snapshotBytes, + captureSnapshot.getUnAppliedEntries(), captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - persistenceProvider.saveSnapshot(sn); + context.getPersistenceProvider().saveSnapshot(snapshot); - LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage()); + LOG.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot.getLogMessage()); - long dataThreshold = Runtime.getRuntime().totalMemory() * + long dataThreshold = totalMemory * context.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. + boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold; + + boolean logSizeExceededSnapshotBatchCount = + context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount(); +LOG.debug("Log size: {}, getSnapshotBatchCount: {}",context.getReplicatedLog().size(),context.getConfigParams().getSnapshotBatchCount()); + if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount) { + if(LOG.isDebugEnabled()) { + if(dataSizeThresholdExceeded) { + LOG.debug("{}: log data size {} exceeds the memory threshold {} - doing snapshotPreCommit with index {}", + context.getId(), context.getReplicatedLog().dataSize(), dataThreshold, + captureSnapshot.getLastAppliedIndex()); + } else { + LOG.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with index {}", + context.getId(), context.getReplicatedLog().size(), + context.getConfigParams().getSnapshotBatchCount(), captureSnapshot.getLastAppliedIndex()); + } + } + + // We either exceeded the memory threshold or the log size exceeded the snapshot batch + // count so, to keep the log memory footprint in check, 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 as entries are replicated to all. context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); - currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex()); + // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an + // install snapshot to a follower. + if(captureSnapshot.getReplicatedToAllIndex() >= 0) { + currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex()); + } + } else if(captureSnapshot.getReplicatedToAllIndex() != -1){ // clear the log based on replicatedToAllIndex context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(), @@ -273,15 +347,14 @@ public class SnapshotManager implements SnapshotState { context.getReplicatedLog().getSnapshotTerm()); } - LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " + - "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(), - captureSnapshot.getLastAppliedTerm()); + LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " + + "and term: {}", context.getId(), context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm()); if (context.getId().equals(currentBehavior.getLeaderId()) && captureSnapshot.isInstallSnapshotInitiated()) { // this would be call straight to the leader and won't initiate in serialization - currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot( - ByteString.copyFrom(snapshotBytes))); + currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(snapshot)); } captureSnapshot = null; @@ -298,26 +371,49 @@ public class SnapshotManager implements SnapshotState { private class Persisting extends AbstractSnapshotState { @Override - public void commit(DataPersistenceProvider persistenceProvider, long sequenceNumber) { - context.getReplicatedLog().snapshotCommit(); - persistenceProvider.deleteSnapshots(new SnapshotSelectionCriteria( + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { + LOG.debug("Snapshot success sequence number:", sequenceNumber); + + if(applySnapshot != null) { + try { + applySnapshotProcedure.apply(applySnapshot.getState()); + + //clears the followers log, sets the snapshot index to ensure adjusted-index works + context.setReplicatedLog(ReplicatedLogImpl.newInstance(applySnapshot, context, currentBehavior)); + context.setLastApplied(applySnapshot.getLastAppliedIndex()); + context.setCommitIndex(applySnapshot.getLastAppliedIndex()); + } catch (Exception e) { + LOG.error("Error applying snapshot", e); + } + } else { + context.getReplicatedLog().snapshotCommit(); + } + + context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria( sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); - persistenceProvider.deleteMessages(sequenceNumber); + context.getPersistenceProvider().deleteMessages(lastSequenceNumber); + lastSequenceNumber = -1; + applySnapshot = null; SnapshotManager.this.currentState = IDLE; } @Override public void rollback() { - context.getReplicatedLog().snapshotRollback(); - - LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." + - "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(), - context.getReplicatedLog().getSnapshotIndex(), - context.getReplicatedLog().getSnapshotTerm(), - context.getReplicatedLog().size()); + // Nothing to rollback if we're applying a snapshot from the leader. + if(applySnapshot == null) { + context.getReplicatedLog().snapshotRollback(); + + LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." + + "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(), + context.getReplicatedLog().getSnapshotIndex(), + context.getReplicatedLog().getSnapshotTerm(), + context.getReplicatedLog().size()); + } + lastSequenceNumber = -1; + applySnapshot = null; SnapshotManager.this.currentState = IDLE; } @@ -333,7 +429,7 @@ public class SnapshotManager implements SnapshotState { long getTerm(); } - private static class LastAppliedTermInformationReader implements TermInformationReader{ + static class LastAppliedTermInformationReader implements TermInformationReader{ private long index; private long term; @@ -344,13 +440,15 @@ public class SnapshotManager implements SnapshotState { this.term = -1L; if (!hasFollowers) { if(lastLogEntry != null) { + // since we have persisted the last-log-entry to persistent journal before the capture, + // we would want to snapshot from this entry. index = lastLogEntry.getIndex(); term = lastLogEntry.getTerm(); } } else if (entry != null) { index = entry.getIndex(); term = entry.getTerm(); - } else if(originalIndex == log.getSnapshotIndex()){ + } else if(log.getSnapshotIndex() > -1){ index = log.getSnapshotIndex(); term = log.getSnapshotTerm(); }