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%2FSnapshotManager.java;h=9571173175ff220aa3fb5b6b9ba757eee150ffe4;hp=9a916625c9331413685d6263bfe053930b6795bf;hb=f1c3050779d7770ef6a12a67a1870765c3dfd9eb;hpb=412db94945c5db5d2da918f5e23bd3abcecc4d10 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 9a916625c9..9571173175 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 @@ -11,10 +11,11 @@ package org.opendaylight.controller.cluster.raft; import akka.japi.Procedure; import akka.persistence.SnapshotSelectionCriteria; import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; import java.util.List; +import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot; import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot; import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot; +import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete; import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; import org.slf4j.Logger; @@ -38,11 +39,18 @@ public class SnapshotManager implements SnapshotState { private Procedure createSnapshotProcedure; + private ApplySnapshot applySnapshot; + private Procedure applySnapshotProcedure; + public SnapshotManager(RaftActorContext context, Logger logger) { this.context = context; this.LOG = logger; } + public boolean isApplying() { + return applySnapshot != null; + } + @Override public boolean isCapturing() { return currentState.isCapturing(); @@ -58,14 +66,19 @@ public class SnapshotManager implements SnapshotState { return currentState.capture(lastLogEntry, replicatedToAllIndex); } + @Override + public void apply(ApplySnapshot snapshot) { + currentState.apply(snapshot); + } + @Override public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { currentState.persist(snapshotBytes, currentBehavior, totalMemory); } @Override - public void commit(long sequenceNumber) { - currentState.commit(sequenceNumber); + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { + currentState.commit(sequenceNumber, currentBehavior); } @Override @@ -82,24 +95,63 @@ public class SnapshotManager implements SnapshotState { 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(){ - return context.getPeerAddresses().keySet().size() > 0; + return context.hasFollowers(); } private String persistenceId(){ return context.getId(); } + public CaptureSnapshot newCaptureSnapshot(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, + boolean installSnapshotInitiated) { + TermInformationReader lastAppliedTermInfoReader = + lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(), + lastLogEntry, hasFollowers()); + + long lastAppliedIndex = lastAppliedTermInfoReader.getIndex(); + long lastAppliedTerm = lastAppliedTermInfoReader.getTerm(); + + TermInformationReader replicatedToAllTermInfoReader = + replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex); + + long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex(); + long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm(); + + List unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1); + + 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); + } + + return new CaptureSnapshot(lastLogEntryIndex, lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm, + newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, installSnapshotInitiated); + } + private class AbstractSnapshotState implements SnapshotState { @Override public boolean isCapturing() { - return false; + return true; } @Override @@ -114,13 +166,18 @@ public class SnapshotManager implements SnapshotState { return false; } + @Override + public void apply(ApplySnapshot snapshot) { + LOG.debug("apply should not be called in state {}", this); + } + @Override public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { LOG.debug("persist should not be called in state {}", this); } @Override - public void commit(long sequenceNumber) { + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { LOG.debug("commit should not be called in state {}", this); } @@ -168,27 +225,13 @@ public class SnapshotManager implements SnapshotState { private class Idle extends AbstractSnapshotState { - private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { - TermInformationReader lastAppliedTermInfoReader = - lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(), - lastLogEntry, hasFollowers()); - - long lastAppliedIndex = lastAppliedTermInfoReader.getIndex(); - long lastAppliedTerm = lastAppliedTermInfoReader.getTerm(); - - TermInformationReader replicatedToAllTermInfoReader = - replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex); - - long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex(); - long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm(); - - // send a CaptureSnapshot to self to make the expensive operation async. - - List unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1); + @Override + public boolean isCapturing() { + return false; + } - captureSnapshot = new CaptureSnapshot(lastLogEntry.getIndex(), - lastLogEntry.getTerm(), lastAppliedIndex, lastAppliedTerm, - newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, targetFollower != null); + private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { + captureSnapshot = newCaptureSnapshot(lastLogEntry, replicatedToAllIndex, targetFollower != null); if(captureSnapshot.isInstallSnapshotInitiated()) { LOG.info("{}: Initiating snapshot capture {} to install on {}", @@ -224,6 +267,19 @@ public class SnapshotManager implements SnapshotState { return capture(lastLogEntry, replicatedToAllIndex, targetFollower); } + @Override + public void apply(ApplySnapshot applySnapshot) { + SnapshotManager.this.applySnapshot = applySnapshot; + + lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber(); + + LOG.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber); + + context.getPersistenceProvider().saveSnapshot(applySnapshot.getSnapshot()); + + SnapshotManager.this.currentState = PERSISTING; + } + @Override public String toString() { return "Idle"; @@ -237,38 +293,46 @@ public class SnapshotManager implements SnapshotState { private class Creating extends AbstractSnapshotState { - @Override - public boolean isCapturing() { - return true; - } - @Override 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, + Snapshot snapshot = Snapshot.create(snapshotBytes, captureSnapshot.getUnAppliedEntries(), captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), - captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm()); + captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm(), + context.getTermInformation().getCurrentTerm(), + context.getTermInformation().getVotedFor()); - context.getPersistenceProvider().saveSnapshot(sn); + context.getPersistenceProvider().saveSnapshot(snapshot); - LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage()); + LOG.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot); long dataThreshold = totalMemory * context.getConfigParams().getSnapshotDataThresholdPercentage() / 100; - if (context.getReplicatedLog().dataSize() > dataThreshold) { + boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold; + boolean logSizeExceededSnapshotBatchCount = + context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount(); + + if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount) { if(LOG.isDebugEnabled()) { - LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}", - persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold, - captureSnapshot.getLastAppliedIndex()); + 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()); + } } - // 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. + // 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()); @@ -293,15 +357,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; @@ -318,28 +381,59 @@ public class SnapshotManager implements SnapshotState { private class Persisting extends AbstractSnapshotState { @Override - public void commit(long sequenceNumber) { - context.getReplicatedLog().snapshotCommit(); + public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { + LOG.debug("Snapshot success sequence number: {}", sequenceNumber); + + if(applySnapshot != null) { + try { + Snapshot snapshot = applySnapshot.getSnapshot(); + applySnapshotProcedure.apply(snapshot.getState()); + + //clears the followers log, sets the snapshot index to ensure adjusted-index works + context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context, currentBehavior)); + context.setLastApplied(snapshot.getLastAppliedIndex()); + context.setCommitIndex(snapshot.getLastAppliedIndex()); + + applySnapshot.getCallback().onSuccess(); + } catch (Exception e) { + LOG.error("Error applying snapshot", e); + } + } else { + context.getReplicatedLog().snapshotCommit(); + } + context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria( sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); context.getPersistenceProvider().deleteMessages(lastSequenceNumber); - lastSequenceNumber = -1; - SnapshotManager.this.currentState = IDLE; + snapshotComplete(); } @Override public void rollback() { - context.getReplicatedLog().snapshotRollback(); + // 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()); + } else { + applySnapshot.getCallback().onFailure(); + } - 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()); + snapshotComplete(); + } + private void snapshotComplete() { + lastSequenceNumber = -1; + applySnapshot = null; SnapshotManager.this.currentState = IDLE; + + context.getActor().tell(SnapshotComplete.INSTANCE, context.getActor()); } @Override @@ -365,6 +459,8 @@ 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(); }