2 * Copyright (c) 2014 Cisco Systems, Inc. and others. All rights reserved.
4 * This program and the accompanying materials are made available under the
5 * terms of the Eclipse Public License v1.0 which accompanies this distribution,
6 * and is available at http://www.eclipse.org/legal/epl-v10.html
9 package org.opendaylight.controller.cluster.raft;
11 import akka.japi.Procedure;
12 import akka.persistence.SnapshotSelectionCriteria;
13 import com.google.common.annotations.VisibleForTesting;
14 import com.google.protobuf.ByteString;
15 import java.util.List;
16 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
17 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
18 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
19 import org.slf4j.Logger;
21 public class SnapshotManager implements SnapshotState {
23 private final SnapshotState IDLE = new Idle();
24 private final SnapshotState PERSISTING = new Persisting();
25 private final SnapshotState CREATING = new Creating();
27 private final Logger LOG;
28 private final RaftActorContext context;
29 private final LastAppliedTermInformationReader lastAppliedTermInformationReader =
30 new LastAppliedTermInformationReader();
31 private final ReplicatedToAllTermInformationReader replicatedToAllTermInformationReader =
32 new ReplicatedToAllTermInformationReader();
35 private SnapshotState currentState = IDLE;
36 private CaptureSnapshot captureSnapshot;
37 private long lastSequenceNumber = -1;
39 private Procedure<Void> createSnapshotProcedure;
41 public SnapshotManager(RaftActorContext context, Logger logger) {
42 this.context = context;
47 public boolean isCapturing() {
48 return currentState.isCapturing();
52 public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
53 return currentState.captureToInstall(lastLogEntry, replicatedToAllIndex, targetFollower);
57 public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
58 return currentState.capture(lastLogEntry, replicatedToAllIndex);
62 public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
63 currentState.persist(snapshotBytes, currentBehavior, totalMemory);
67 public void commit(long sequenceNumber) {
68 currentState.commit(sequenceNumber);
72 public void rollback() {
73 currentState.rollback();
77 public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
78 return currentState.trimLog(desiredTrimIndex, currentBehavior);
81 public void setCreateSnapshotCallable(Procedure<Void> createSnapshotProcedure) {
82 this.createSnapshotProcedure = createSnapshotProcedure;
85 public long getLastSequenceNumber() {
86 return lastSequenceNumber;
90 public CaptureSnapshot getCaptureSnapshot() {
91 return captureSnapshot;
94 private boolean hasFollowers(){
95 return context.getPeerAddresses().keySet().size() > 0;
98 private String persistenceId(){
99 return context.getId();
102 private class AbstractSnapshotState implements SnapshotState {
105 public boolean isCapturing() {
110 public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
111 LOG.debug("capture should not be called in state {}", this);
116 public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
117 LOG.debug("captureToInstall should not be called in state {}", this);
122 public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
123 LOG.debug("persist should not be called in state {}", this);
127 public void commit(long sequenceNumber) {
128 LOG.debug("commit should not be called in state {}", this);
132 public void rollback() {
133 LOG.debug("rollback should not be called in state {}", this);
137 public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
138 LOG.debug("trimLog should not be called in state {}", this);
142 protected long doTrimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior){
143 // we would want to keep the lastApplied as its used while capturing snapshots
144 long lastApplied = context.getLastApplied();
145 long tempMin = Math.min(desiredTrimIndex, (lastApplied > -1 ? lastApplied - 1 : -1));
147 if(LOG.isTraceEnabled()) {
148 LOG.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, tempMin: {}",
149 persistenceId(), desiredTrimIndex, lastApplied, tempMin);
152 if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) {
153 LOG.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin,
154 context.getTermInformation().getCurrentTerm());
156 //use the term of the temp-min, since we check for isPresent, entry will not be null
157 ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
158 context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
159 context.getReplicatedLog().snapshotCommit();
161 } else if(tempMin > currentBehavior.getReplicatedToAllIndex()) {
162 // It's possible a follower was lagging and an install snapshot advanced its match index past
163 // the current replicatedToAllIndex. Since the follower is now caught up we should advance the
164 // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely
165 // due to a previous snapshot triggered by the memory threshold exceeded, in that case we
166 // trim the log to the last applied index even if previous entries weren't replicated to all followers.
167 currentBehavior.setReplicatedToAllIndex(tempMin);
173 private class Idle extends AbstractSnapshotState {
175 private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
176 TermInformationReader lastAppliedTermInfoReader =
177 lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(),
178 lastLogEntry, hasFollowers());
180 long lastAppliedIndex = lastAppliedTermInfoReader.getIndex();
181 long lastAppliedTerm = lastAppliedTermInfoReader.getTerm();
183 TermInformationReader replicatedToAllTermInfoReader =
184 replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex);
186 long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex();
187 long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm();
189 // send a CaptureSnapshot to self to make the expensive operation async.
191 List<ReplicatedLogEntry> unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1);
193 captureSnapshot = new CaptureSnapshot(lastLogEntry.getIndex(),
194 lastLogEntry.getTerm(), lastAppliedIndex, lastAppliedTerm,
195 newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, targetFollower != null);
197 if(captureSnapshot.isInstallSnapshotInitiated()) {
198 LOG.info("{}: Initiating snapshot capture {} to install on {}",
199 persistenceId(), captureSnapshot, targetFollower);
201 LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
204 lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
206 LOG.debug("lastSequenceNumber prior to capture: {}", lastSequenceNumber);
208 SnapshotManager.this.currentState = CREATING;
211 createSnapshotProcedure.apply(null);
212 } catch (Exception e) {
213 SnapshotManager.this.currentState = IDLE;
214 LOG.error("Error creating snapshot", e);
222 public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
223 return capture(lastLogEntry, replicatedToAllIndex, null);
227 public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
228 return capture(lastLogEntry, replicatedToAllIndex, targetFollower);
232 public String toString() {
237 public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
238 return doTrimLog(desiredTrimIndex, currentBehavior);
242 private class Creating extends AbstractSnapshotState {
245 public boolean isCapturing() {
250 public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
251 // create a snapshot object from the state provided and save it
252 // when snapshot is saved async, SaveSnapshotSuccess is raised.
254 Snapshot sn = Snapshot.create(snapshotBytes,
255 captureSnapshot.getUnAppliedEntries(),
256 captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
257 captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
259 context.getPersistenceProvider().saveSnapshot(sn);
261 LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
263 long dataThreshold = totalMemory *
264 context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
265 if (context.getReplicatedLog().dataSize() > dataThreshold) {
267 if(LOG.isDebugEnabled()) {
268 LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}",
269 persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold,
270 captureSnapshot.getLastAppliedIndex());
273 // if memory is less, clear the log based on lastApplied.
274 // this could/should only happen if one of the followers is down
275 // as normally we keep removing from the log when its replicated to all.
276 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
277 captureSnapshot.getLastAppliedTerm());
279 // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
280 // install snapshot to a follower.
281 if(captureSnapshot.getReplicatedToAllIndex() >= 0) {
282 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
285 } else if(captureSnapshot.getReplicatedToAllIndex() != -1){
286 // clear the log based on replicatedToAllIndex
287 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
288 captureSnapshot.getReplicatedToAllTerm());
290 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
292 // The replicatedToAllIndex was not found in the log
293 // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
294 // In this scenario we may need to save the snapshot to the akka persistence
295 // snapshot for recovery but we do not need to do the replicated log trimming.
296 context.getReplicatedLog().snapshotPreCommit(context.getReplicatedLog().getSnapshotIndex(),
297 context.getReplicatedLog().getSnapshotTerm());
300 LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
301 "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
302 captureSnapshot.getLastAppliedTerm());
304 if (context.getId().equals(currentBehavior.getLeaderId())
305 && captureSnapshot.isInstallSnapshotInitiated()) {
306 // this would be call straight to the leader and won't initiate in serialization
307 currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(
308 ByteString.copyFrom(snapshotBytes)));
311 captureSnapshot = null;
312 SnapshotManager.this.currentState = PERSISTING;
316 public String toString() {
322 private class Persisting extends AbstractSnapshotState {
325 public void commit(long sequenceNumber) {
326 LOG.debug("Snapshot success sequence number:", sequenceNumber);
327 context.getReplicatedLog().snapshotCommit();
328 context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(
329 sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
331 context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
333 lastSequenceNumber = -1;
334 SnapshotManager.this.currentState = IDLE;
338 public void rollback() {
339 context.getReplicatedLog().snapshotRollback();
341 LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." +
342 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
343 context.getReplicatedLog().getSnapshotIndex(),
344 context.getReplicatedLog().getSnapshotTerm(),
345 context.getReplicatedLog().size());
347 SnapshotManager.this.currentState = IDLE;
351 public String toString() {
357 private static interface TermInformationReader {
362 static class LastAppliedTermInformationReader implements TermInformationReader{
366 public LastAppliedTermInformationReader init(ReplicatedLog log, long originalIndex,
367 ReplicatedLogEntry lastLogEntry, boolean hasFollowers){
368 ReplicatedLogEntry entry = log.get(originalIndex);
372 if(lastLogEntry != null) {
373 // since we have persisted the last-log-entry to persistent journal before the capture,
374 // we would want to snapshot from this entry.
375 index = lastLogEntry.getIndex();
376 term = lastLogEntry.getTerm();
378 } else if (entry != null) {
379 index = entry.getIndex();
380 term = entry.getTerm();
381 } else if(log.getSnapshotIndex() > -1){
382 index = log.getSnapshotIndex();
383 term = log.getSnapshotTerm();
389 public long getIndex(){
394 public long getTerm(){
399 private static class ReplicatedToAllTermInformationReader implements TermInformationReader{
403 ReplicatedToAllTermInformationReader init(ReplicatedLog log, long originalIndex){
404 ReplicatedLogEntry entry = log.get(originalIndex);
409 index = entry.getIndex();
410 term = entry.getTerm();
417 public long getIndex(){
422 public long getTerm(){