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;
86 public CaptureSnapshot getCaptureSnapshot() {
87 return captureSnapshot;
90 private boolean hasFollowers(){
91 return context.getPeerAddresses().keySet().size() > 0;
94 private String persistenceId(){
95 return context.getId();
98 private class AbstractSnapshotState implements SnapshotState {
101 public boolean isCapturing() {
106 public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
107 LOG.debug("capture should not be called in state {}", this);
112 public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
113 LOG.debug("captureToInstall should not be called in state {}", this);
118 public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
119 LOG.debug("persist should not be called in state {}", this);
123 public void commit(long sequenceNumber) {
124 LOG.debug("commit should not be called in state {}", this);
128 public void rollback() {
129 LOG.debug("rollback should not be called in state {}", this);
133 public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
134 LOG.debug("trimLog should not be called in state {}", this);
138 protected long doTrimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior){
139 // we would want to keep the lastApplied as its used while capturing snapshots
140 long lastApplied = context.getLastApplied();
141 long tempMin = Math.min(desiredTrimIndex, (lastApplied > -1 ? lastApplied - 1 : -1));
143 if(LOG.isTraceEnabled()) {
144 LOG.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, tempMin: {}",
145 persistenceId(), desiredTrimIndex, lastApplied, tempMin);
148 if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) {
149 LOG.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin,
150 context.getTermInformation().getCurrentTerm());
152 //use the term of the temp-min, since we check for isPresent, entry will not be null
153 ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
154 context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
155 context.getReplicatedLog().snapshotCommit();
157 } else if(tempMin > currentBehavior.getReplicatedToAllIndex()) {
158 // It's possible a follower was lagging and an install snapshot advanced its match index past
159 // the current replicatedToAllIndex. Since the follower is now caught up we should advance the
160 // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely
161 // due to a previous snapshot triggered by the memory threshold exceeded, in that case we
162 // trim the log to the last applied index even if previous entries weren't replicated to all followers.
163 currentBehavior.setReplicatedToAllIndex(tempMin);
169 private class Idle extends AbstractSnapshotState {
171 private boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
172 TermInformationReader lastAppliedTermInfoReader =
173 lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(),
174 lastLogEntry, hasFollowers());
176 long lastAppliedIndex = lastAppliedTermInfoReader.getIndex();
177 long lastAppliedTerm = lastAppliedTermInfoReader.getTerm();
179 TermInformationReader replicatedToAllTermInfoReader =
180 replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex);
182 long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex();
183 long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm();
185 // send a CaptureSnapshot to self to make the expensive operation async.
187 List<ReplicatedLogEntry> unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1);
189 captureSnapshot = new CaptureSnapshot(lastLogEntry.getIndex(),
190 lastLogEntry.getTerm(), lastAppliedIndex, lastAppliedTerm,
191 newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, targetFollower != null);
193 if(captureSnapshot.isInstallSnapshotInitiated()) {
194 LOG.info("{}: Initiating snapshot capture {} to install on {}",
195 persistenceId(), captureSnapshot, targetFollower);
197 LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
200 lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
202 LOG.debug("lastSequenceNumber prior to capture: {}", lastSequenceNumber);
205 createSnapshotProcedure.apply(null);
206 } catch (Exception e) {
207 LOG.error("Error creating snapshot", e);
211 SnapshotManager.this.currentState = CREATING;
216 public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
217 return capture(lastLogEntry, replicatedToAllIndex, null);
221 public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
222 return capture(lastLogEntry, replicatedToAllIndex, targetFollower);
226 public String toString() {
231 public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
232 return doTrimLog(desiredTrimIndex, currentBehavior);
236 private class Creating extends AbstractSnapshotState {
239 public boolean isCapturing() {
244 public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
245 // create a snapshot object from the state provided and save it
246 // when snapshot is saved async, SaveSnapshotSuccess is raised.
248 Snapshot sn = Snapshot.create(snapshotBytes,
249 captureSnapshot.getUnAppliedEntries(),
250 captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
251 captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
253 context.getPersistenceProvider().saveSnapshot(sn);
255 LOG.info("{}: Persisting of snapshot done:{}", persistenceId(), sn.getLogMessage());
257 long dataThreshold = totalMemory *
258 context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
259 if (context.getReplicatedLog().dataSize() > dataThreshold) {
261 if(LOG.isDebugEnabled()) {
262 LOG.debug("{}: dataSize {} exceeds dataThreshold {} - doing snapshotPreCommit with index {}",
263 persistenceId(), context.getReplicatedLog().dataSize(), dataThreshold,
264 captureSnapshot.getLastAppliedIndex());
267 // if memory is less, clear the log based on lastApplied.
268 // this could/should only happen if one of the followers is down
269 // as normally we keep removing from the log when its replicated to all.
270 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
271 captureSnapshot.getLastAppliedTerm());
273 // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
274 // install snapshot to a follower.
275 if(captureSnapshot.getReplicatedToAllIndex() >= 0) {
276 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
279 } else if(captureSnapshot.getReplicatedToAllIndex() != -1){
280 // clear the log based on replicatedToAllIndex
281 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
282 captureSnapshot.getReplicatedToAllTerm());
284 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
286 // The replicatedToAllIndex was not found in the log
287 // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
288 // In this scenario we may need to save the snapshot to the akka persistence
289 // snapshot for recovery but we do not need to do the replicated log trimming.
290 context.getReplicatedLog().snapshotPreCommit(context.getReplicatedLog().getSnapshotIndex(),
291 context.getReplicatedLog().getSnapshotTerm());
294 LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex:{} " +
295 "and term:{}", persistenceId(), captureSnapshot.getLastAppliedIndex(),
296 captureSnapshot.getLastAppliedTerm());
298 if (context.getId().equals(currentBehavior.getLeaderId())
299 && captureSnapshot.isInstallSnapshotInitiated()) {
300 // this would be call straight to the leader and won't initiate in serialization
301 currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(
302 ByteString.copyFrom(snapshotBytes)));
305 captureSnapshot = null;
306 SnapshotManager.this.currentState = PERSISTING;
310 public String toString() {
316 private class Persisting extends AbstractSnapshotState {
319 public void commit(long sequenceNumber) {
320 context.getReplicatedLog().snapshotCommit();
321 context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(
322 sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
324 context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
326 lastSequenceNumber = -1;
327 SnapshotManager.this.currentState = IDLE;
331 public void rollback() {
332 context.getReplicatedLog().snapshotRollback();
334 LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." +
335 "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
336 context.getReplicatedLog().getSnapshotIndex(),
337 context.getReplicatedLog().getSnapshotTerm(),
338 context.getReplicatedLog().size());
340 SnapshotManager.this.currentState = IDLE;
344 public String toString() {
350 private static interface TermInformationReader {
355 static class LastAppliedTermInformationReader implements TermInformationReader{
359 public LastAppliedTermInformationReader init(ReplicatedLog log, long originalIndex,
360 ReplicatedLogEntry lastLogEntry, boolean hasFollowers){
361 ReplicatedLogEntry entry = log.get(originalIndex);
365 if(lastLogEntry != null) {
366 index = lastLogEntry.getIndex();
367 term = lastLogEntry.getTerm();
369 } else if (entry != null) {
370 index = entry.getIndex();
371 term = entry.getTerm();
372 } else if(log.getSnapshotIndex() > -1){
373 index = log.getSnapshotIndex();
374 term = log.getSnapshotTerm();
380 public long getIndex(){
385 public long getTerm(){
390 private static class ReplicatedToAllTermInformationReader implements TermInformationReader{
394 ReplicatedToAllTermInformationReader init(ReplicatedLog log, long originalIndex){
395 ReplicatedLogEntry entry = log.get(originalIndex);
400 index = entry.getIndex();
401 term = entry.getTerm();
408 public long getIndex(){
413 public long getTerm(){