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=79f2ce9b4ca0fb2b0deca496eed67e069bc1a547;hp=9571173175ff220aa3fb5b6b9ba757eee150ffe4;hb=b0f8283587b5cc8573d29f66219cbe7f70e21e1b;hpb=f1c3050779d7770ef6a12a67a1870765c3dfd9eb 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 9571173175..79f2ce9b4c 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 @@ -8,10 +8,10 @@ package org.opendaylight.controller.cluster.raft; -import akka.japi.Procedure; import akka.persistence.SnapshotSelectionCriteria; import com.google.common.annotations.VisibleForTesting; import java.util.List; +import java.util.function.Consumer; 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; @@ -19,13 +19,24 @@ import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete; import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior; import org.slf4j.Logger; +/** + * Manages the capturing of snapshots for a RaftActor. + * + * @author Moiz Raja + * @author Thomas Pantelis + */ public class SnapshotManager implements SnapshotState { + @SuppressWarnings("checkstyle:MemberName") private final SnapshotState IDLE = new Idle(); + + @SuppressWarnings({"checkstyle:MemberName", "checkstyle:AbbreviationAsWordInName"}) private final SnapshotState PERSISTING = new Persisting(); + + @SuppressWarnings({"checkstyle:MemberName", "checkstyle:AbbreviationAsWordInName"}) private final SnapshotState CREATING = new Creating(); - private final Logger LOG; + private final Logger log; private final RaftActorContext context; private final LastAppliedTermInformationReader lastAppliedTermInformationReader = new LastAppliedTermInformationReader(); @@ -37,14 +48,20 @@ public class SnapshotManager implements SnapshotState { private CaptureSnapshot captureSnapshot; private long lastSequenceNumber = -1; - private Procedure createSnapshotProcedure; + private Runnable createSnapshotProcedure; private ApplySnapshot applySnapshot; - private Procedure applySnapshotProcedure; - + private Consumer applySnapshotProcedure; + + /** + * Constructs an instance. + * + * @param context the RaftActorContext + * @param logger the Logger + */ public SnapshotManager(RaftActorContext context, Logger logger) { this.context = context; - this.LOG = logger; + this.log = logger; } public boolean isApplying() { @@ -72,13 +89,13 @@ public class SnapshotManager implements SnapshotState { } @Override - public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { - currentState.persist(snapshotBytes, currentBehavior, totalMemory); + public void persist(final byte[] snapshotBytes, final long totalMemory) { + currentState.persist(snapshotBytes, totalMemory); } @Override - public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { - currentState.commit(sequenceNumber, currentBehavior); + public void commit(final long sequenceNumber, long timeStamp) { + currentState.commit(sequenceNumber, timeStamp); } @Override @@ -87,15 +104,15 @@ public class SnapshotManager implements SnapshotState { } @Override - public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { - return currentState.trimLog(desiredTrimIndex, currentBehavior); + public long trimLog(final long desiredTrimIndex) { + return currentState.trimLog(desiredTrimIndex); } - public void setCreateSnapshotCallable(Procedure createSnapshotProcedure) { + public void setCreateSnapshotRunnable(Runnable createSnapshotProcedure) { this.createSnapshotProcedure = createSnapshotProcedure; } - public void setApplySnapshotProcedure(Procedure applySnapshotProcedure) { + public void setApplySnapshotConsumer(Consumer applySnapshotProcedure) { this.applySnapshotProcedure = applySnapshotProcedure; } @@ -108,14 +125,22 @@ public class SnapshotManager implements SnapshotState { return captureSnapshot; } - private boolean hasFollowers(){ + private boolean hasFollowers() { return context.hasFollowers(); } - private String persistenceId(){ + private String persistenceId() { return context.getId(); } + /** + * Constructs a CaptureSnapshot instance. + * + * @param lastLogEntry the last log entry for the snapshot. + * @param replicatedToAllIndex the index of the last entry replicated to all followers. + * @param installSnapshotInitiated true if snapshot is initiated to install on a follower. + * @return a new CaptureSnapshot instance. + */ public CaptureSnapshot newCaptureSnapshot(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, boolean installSnapshotInitiated) { TermInformationReader lastAppliedTermInfoReader = @@ -135,12 +160,12 @@ public class SnapshotManager implements SnapshotState { long lastLogEntryIndex = lastAppliedIndex; long lastLogEntryTerm = lastAppliedTerm; - if(lastLogEntry != null) { + if (lastLogEntry != null) { lastLogEntryIndex = lastLogEntry.getIndex(); lastLogEntryTerm = lastLogEntry.getTerm(); } else { - LOG.warn("Capturing Snapshot : lastLogEntry is null. Using lastAppliedIndex {} and lastAppliedTerm {} instead.", - lastAppliedIndex, lastAppliedTerm); + log.debug("{}: Capturing Snapshot : lastLogEntry is null. Using lastAppliedIndex {} and " + + "lastAppliedTerm {} instead.", persistenceId(), lastAppliedIndex, lastAppliedTerm); } return new CaptureSnapshot(lastLogEntryIndex, lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm, @@ -156,54 +181,55 @@ public class SnapshotManager implements SnapshotState { @Override public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) { - LOG.debug("capture should not be called in state {}", this); + log.debug("capture should not be called in state {}", this); return false; } @Override - public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { - LOG.debug("captureToInstall should not be called in state {}", this); + public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, + String targetFollower) { + log.debug("captureToInstall should not be called in state {}", this); return false; } @Override public void apply(ApplySnapshot snapshot) { - LOG.debug("apply should not be called in state {}", this); + 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); + public void persist(final byte[] snapshotBytes, final long totalMemory) { + log.debug("persist should not be called in state {}", this); } @Override - public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { - LOG.debug("commit should not be called in state {}", this); + public void commit(final long sequenceNumber, long timeStamp) { + log.debug("commit should not be called in state {}", this); } @Override public void rollback() { - LOG.debug("rollback should not be called in state {}", this); + log.debug("rollback should not be called in state {}", this); } @Override - public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { - LOG.debug("trimLog should not be called in state {}", this); + public long trimLog(final long desiredTrimIndex) { + log.debug("trimLog should not be called in state {}", this); return -1; } - protected long doTrimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior){ + protected long doTrimLog(final long desiredTrimIndex) { // 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)); + long tempMin = Math.min(desiredTrimIndex, lastApplied > -1 ? lastApplied - 1 : -1); - if(LOG.isTraceEnabled()) { - LOG.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, 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, + log.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin, context.getTermInformation().getCurrentTerm()); //use the term of the temp-min, since we check for isPresent, entry will not be null @@ -211,7 +237,10 @@ public class SnapshotManager implements SnapshotState { context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm()); context.getReplicatedLog().snapshotCommit(); return tempMin; - } else if(tempMin > currentBehavior.getReplicatedToAllIndex()) { + } + + final RaftActorBehavior currentBehavior = context.getCurrentBehavior(); + 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 @@ -230,27 +259,28 @@ public class SnapshotManager implements SnapshotState { return false; } + @SuppressWarnings("checkstyle:IllegalCatch") 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 {}", + if (captureSnapshot.isInstallSnapshotInitiated()) { + log.info("{}: Initiating snapshot capture {} to install on {}", persistenceId(), captureSnapshot, targetFollower); } else { - LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot); + log.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot); } lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber(); - LOG.debug("lastSequenceNumber prior to capture: {}", lastSequenceNumber); + log.debug("{}: lastSequenceNumber prior to capture: {}", persistenceId(), lastSequenceNumber); SnapshotManager.this.currentState = CREATING; try { - createSnapshotProcedure.apply(null); + createSnapshotProcedure.run(); } catch (Exception e) { SnapshotManager.this.currentState = IDLE; - LOG.error("Error creating snapshot", e); + log.error("Error creating snapshot", e); return false; } @@ -263,19 +293,20 @@ public class SnapshotManager implements SnapshotState { } @Override - public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) { + public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, + String targetFollower) { return capture(lastLogEntry, replicatedToAllIndex, targetFollower); } @Override - public void apply(ApplySnapshot applySnapshot) { - SnapshotManager.this.applySnapshot = applySnapshot; + public void apply(ApplySnapshot toApply) { + SnapshotManager.this.applySnapshot = toApply; lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber(); - LOG.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber); + log.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber); - context.getPersistenceProvider().saveSnapshot(applySnapshot.getSnapshot()); + context.getPersistenceProvider().saveSnapshot(toApply.getSnapshot()); SnapshotManager.this.currentState = PERSISTING; } @@ -286,15 +317,15 @@ public class SnapshotManager implements SnapshotState { } @Override - public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) { - return doTrimLog(desiredTrimIndex, currentBehavior); + public long trimLog(final long desiredTrimIndex) { + return doTrimLog(desiredTrimIndex); } } private class Creating extends AbstractSnapshotState { @Override - public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) { + public void persist(final byte[] snapshotBytes, final long totalMemory) { // create a snapshot object from the state provided and save it // when snapshot is saved async, SaveSnapshotSuccess is raised. @@ -303,29 +334,30 @@ public class SnapshotManager implements SnapshotState { captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(), captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm(), context.getTermInformation().getCurrentTerm(), - context.getTermInformation().getVotedFor()); + context.getTermInformation().getVotedFor(), context.getPeerServerInfo(true)); context.getPersistenceProvider().saveSnapshot(snapshot); - LOG.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot); + log.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot); - long dataThreshold = totalMemory * - context.getConfigParams().getSnapshotDataThresholdPercentage() / 100; + long dataThreshold = totalMemory * context.getConfigParams().getSnapshotDataThresholdPercentage() / 100; boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold; boolean logSizeExceededSnapshotBatchCount = context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount(); + final RaftActorBehavior currentBehavior = context.getCurrentBehavior(); 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()); + 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()); + log.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with " + + "index {}", context.getId(), context.getReplicatedLog().size(), + context.getConfigParams().getSnapshotBatchCount(), + captureSnapshot.getLastAppliedIndex()); } } @@ -338,11 +370,11 @@ public class SnapshotManager implements SnapshotState { // 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) { + if (captureSnapshot.getReplicatedToAllIndex() >= 0) { currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex()); } - } else if(captureSnapshot.getReplicatedToAllIndex() != -1){ + } else if (captureSnapshot.getReplicatedToAllIndex() != -1) { // clear the log based on replicatedToAllIndex context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(), captureSnapshot.getReplicatedToAllTerm()); @@ -357,8 +389,8 @@ public class SnapshotManager implements SnapshotState { context.getReplicatedLog().getSnapshotTerm()); } - LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " + - "and term: {}", context.getId(), context.getReplicatedLog().getSnapshotIndex(), + 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()) @@ -381,29 +413,38 @@ public class SnapshotManager implements SnapshotState { private class Persisting extends AbstractSnapshotState { @Override - public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) { - LOG.debug("Snapshot success sequence number: {}", sequenceNumber); + @SuppressWarnings("checkstyle:IllegalCatch") + public void commit(final long sequenceNumber, long timeStamp) { + log.debug("{}: Snapshot success - sequence number: {}", persistenceId(), sequenceNumber); - if(applySnapshot != null) { + 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.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context)); context.setLastApplied(snapshot.getLastAppliedIndex()); context.setCommitIndex(snapshot.getLastAppliedIndex()); + context.getTermInformation().update(snapshot.getElectionTerm(), snapshot.getElectionVotedFor()); + + if (snapshot.getServerConfiguration() != null) { + context.updatePeerIds(snapshot.getServerConfiguration()); + } + + if (snapshot.getState().length > 0 ) { + applySnapshotProcedure.accept(snapshot.getState()); + } applySnapshot.getCallback().onSuccess(); } catch (Exception e) { - LOG.error("Error applying snapshot", e); + log.error("{}: Error applying snapshot", context.getId(), e); } } else { context.getReplicatedLog().snapshotCommit(); } - context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria( - sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000)); + context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(sequenceNumber, + timeStamp - 1, 0L, 0L)); context.getPersistenceProvider().deleteMessages(lastSequenceNumber); @@ -413,11 +454,11 @@ public class SnapshotManager implements SnapshotState { @Override public void rollback() { // Nothing to rollback if we're applying a snapshot from the leader. - if(applySnapshot == null) { + 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(), + 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()); @@ -443,22 +484,23 @@ public class SnapshotManager implements SnapshotState { } - private static interface TermInformationReader { + private interface TermInformationReader { long getIndex(); + long getTerm(); } - static class LastAppliedTermInformationReader implements TermInformationReader{ + static class LastAppliedTermInformationReader implements TermInformationReader { private long index; private long term; - public LastAppliedTermInformationReader init(ReplicatedLog log, long originalIndex, - ReplicatedLogEntry lastLogEntry, boolean hasFollowers){ + LastAppliedTermInformationReader init(ReplicatedLog log, long originalIndex, ReplicatedLogEntry lastLogEntry, + boolean hasFollowers) { ReplicatedLogEntry entry = log.get(originalIndex); this.index = -1L; this.term = -1L; if (!hasFollowers) { - if(lastLogEntry != null) { + 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(); @@ -467,7 +509,7 @@ public class SnapshotManager implements SnapshotState { } else if (entry != null) { index = entry.getIndex(); term = entry.getTerm(); - } else if(log.getSnapshotIndex() > -1){ + } else if (log.getSnapshotIndex() > -1) { index = log.getSnapshotIndex(); term = log.getSnapshotTerm(); } @@ -475,21 +517,21 @@ public class SnapshotManager implements SnapshotState { } @Override - public long getIndex(){ + public long getIndex() { return this.index; } @Override - public long getTerm(){ + public long getTerm() { return this.term; } } - private static class ReplicatedToAllTermInformationReader implements TermInformationReader{ + private static class ReplicatedToAllTermInformationReader implements TermInformationReader { private long index; private long term; - ReplicatedToAllTermInformationReader init(ReplicatedLog log, long originalIndex){ + ReplicatedToAllTermInformationReader init(ReplicatedLog log, long originalIndex) { ReplicatedLogEntry entry = log.get(originalIndex); this.index = -1L; this.term = -1L; @@ -503,12 +545,12 @@ public class SnapshotManager implements SnapshotState { } @Override - public long getIndex(){ + public long getIndex() { return this.index; } @Override - public long getTerm(){ + public long getTerm() { return this.term; } }