Add direct in-memory journal threshold
[controller.git] / opendaylight / md-sal / sal-akka-raft / src / main / java / org / opendaylight / controller / cluster / raft / SnapshotManager.java
1 /*
2  * Copyright (c) 2014 Cisco Systems, Inc. and others.  All rights reserved.
3  *
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
7  */
8 package org.opendaylight.controller.cluster.raft;
9
10 import akka.persistence.SnapshotSelectionCriteria;
11 import com.google.common.annotations.VisibleForTesting;
12 import com.google.common.io.ByteSource;
13 import java.io.IOException;
14 import java.io.OutputStream;
15 import java.util.List;
16 import java.util.Optional;
17 import java.util.function.Consumer;
18 import org.eclipse.jdt.annotation.NonNull;
19 import org.opendaylight.controller.cluster.io.FileBackedOutputStream;
20 import org.opendaylight.controller.cluster.raft.base.messages.ApplySnapshot;
21 import org.opendaylight.controller.cluster.raft.base.messages.CaptureSnapshot;
22 import org.opendaylight.controller.cluster.raft.base.messages.SendInstallSnapshot;
23 import org.opendaylight.controller.cluster.raft.base.messages.SnapshotComplete;
24 import org.opendaylight.controller.cluster.raft.behaviors.RaftActorBehavior;
25 import org.opendaylight.controller.cluster.raft.persisted.EmptyState;
26 import org.opendaylight.controller.cluster.raft.persisted.Snapshot;
27 import org.slf4j.Logger;
28
29 /**
30  * Manages the capturing of snapshots for a RaftActor.
31  *
32  * @author Moiz Raja
33  * @author Thomas Pantelis
34  */
35 public class SnapshotManager implements SnapshotState {
36
37     @SuppressWarnings("checkstyle:MemberName")
38     private final SnapshotState IDLE = new Idle();
39
40     @SuppressWarnings({"checkstyle:MemberName", "checkstyle:AbbreviationAsWordInName"})
41     private final SnapshotState PERSISTING = new Persisting();
42
43     @SuppressWarnings({"checkstyle:MemberName", "checkstyle:AbbreviationAsWordInName"})
44     private final SnapshotState CREATING = new Creating();
45
46     private final Logger log;
47     private final RaftActorContext context;
48     private final LastAppliedTermInformationReader lastAppliedTermInformationReader =
49             new LastAppliedTermInformationReader();
50     private final ReplicatedToAllTermInformationReader replicatedToAllTermInformationReader =
51             new ReplicatedToAllTermInformationReader();
52
53
54     private SnapshotState currentState = IDLE;
55     private CaptureSnapshot captureSnapshot;
56     private long lastSequenceNumber = -1;
57
58     private Consumer<Optional<OutputStream>> createSnapshotProcedure = null;
59
60     private ApplySnapshot applySnapshot;
61     private RaftActorSnapshotCohort snapshotCohort = NoopRaftActorSnapshotCohort.INSTANCE;
62
63     /**
64      * Constructs an instance.
65      *
66      * @param context the RaftActorContext
67      * @param logger the Logger
68      */
69     public SnapshotManager(final RaftActorContext context, final Logger logger) {
70         this.context = context;
71         this.log = logger;
72     }
73
74     public boolean isApplying() {
75         return applySnapshot != null;
76     }
77
78     @Override
79     public boolean isCapturing() {
80         return currentState.isCapturing();
81     }
82
83     @Override
84     public boolean captureToInstall(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
85             final String targetFollower) {
86         return currentState.captureToInstall(lastLogEntry, replicatedToAllIndex, targetFollower);
87     }
88
89     @Override
90     public boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
91         return currentState.capture(lastLogEntry, replicatedToAllIndex);
92     }
93
94     @Override
95     public boolean captureWithForcedTrim(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
96         return currentState.captureWithForcedTrim(lastLogEntry, replicatedToAllIndex);
97     }
98
99     @Override
100     public void apply(final ApplySnapshot snapshot) {
101         currentState.apply(snapshot);
102     }
103
104     @Override
105     public void persist(final Snapshot.State state, final Optional<OutputStream> installSnapshotStream,
106             final long totalMemory) {
107         currentState.persist(state, installSnapshotStream, totalMemory);
108     }
109
110     @Override
111     public void commit(final long sequenceNumber, final long timeStamp) {
112         currentState.commit(sequenceNumber, timeStamp);
113     }
114
115     @Override
116     public void rollback() {
117         currentState.rollback();
118     }
119
120     @Override
121     public long trimLog(final long desiredTrimIndex) {
122         return currentState.trimLog(desiredTrimIndex);
123     }
124
125     @SuppressWarnings("checkstyle:hiddenField")
126     void setCreateSnapshotConsumer(final Consumer<Optional<OutputStream>> createSnapshotProcedure) {
127         this.createSnapshotProcedure = createSnapshotProcedure;
128     }
129
130     void setSnapshotCohort(final RaftActorSnapshotCohort snapshotCohort) {
131         this.snapshotCohort = snapshotCohort;
132     }
133
134     public Snapshot.@NonNull State convertSnapshot(final ByteSource snapshotBytes) throws IOException {
135         return snapshotCohort.deserializeSnapshot(snapshotBytes);
136     }
137
138     public long getLastSequenceNumber() {
139         return lastSequenceNumber;
140     }
141
142     @VisibleForTesting
143     public CaptureSnapshot getCaptureSnapshot() {
144         return captureSnapshot;
145     }
146
147     private boolean hasFollowers() {
148         return context.hasFollowers();
149     }
150
151     private String persistenceId() {
152         return context.getId();
153     }
154
155     /**
156      * Constructs a CaptureSnapshot instance.
157      *
158      * @param lastLogEntry the last log entry for the snapshot.
159      * @param replicatedToAllIndex the index of the last entry replicated to all followers.
160      * @return a new CaptureSnapshot instance.
161      */
162     public CaptureSnapshot newCaptureSnapshot(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
163                                               final boolean mandatoryTrim) {
164         TermInformationReader lastAppliedTermInfoReader =
165                 lastAppliedTermInformationReader.init(context.getReplicatedLog(), context.getLastApplied(),
166                         lastLogEntry, hasFollowers());
167
168         long lastAppliedIndex = lastAppliedTermInfoReader.getIndex();
169         long lastAppliedTerm = lastAppliedTermInfoReader.getTerm();
170
171         TermInformationReader replicatedToAllTermInfoReader =
172                 replicatedToAllTermInformationReader.init(context.getReplicatedLog(), replicatedToAllIndex);
173
174         long newReplicatedToAllIndex = replicatedToAllTermInfoReader.getIndex();
175         long newReplicatedToAllTerm = replicatedToAllTermInfoReader.getTerm();
176
177         List<ReplicatedLogEntry> unAppliedEntries = context.getReplicatedLog().getFrom(lastAppliedIndex + 1);
178
179         final long lastLogEntryIndex;
180         final long lastLogEntryTerm;
181         if (lastLogEntry == null) {
182             // When we don't have journal present, for example two captureSnapshots executed right after another with no
183             // new journal we still want to preserve the index and term in the snapshot.
184             lastAppliedIndex = lastLogEntryIndex = context.getReplicatedLog().getSnapshotIndex();
185             lastAppliedTerm = lastLogEntryTerm = context.getReplicatedLog().getSnapshotTerm();
186
187             log.debug("{}: Capturing Snapshot : lastLogEntry is null. Using snapshot values lastAppliedIndex {} and "
188                     + "lastAppliedTerm {} instead.", persistenceId(), lastAppliedIndex, lastAppliedTerm);
189         } else {
190             lastLogEntryIndex = lastLogEntry.getIndex();
191             lastLogEntryTerm = lastLogEntry.getTerm();
192         }
193
194         return new CaptureSnapshot(lastLogEntryIndex, lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm,
195                 newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, mandatoryTrim);
196     }
197
198     private class AbstractSnapshotState implements SnapshotState {
199
200         @Override
201         public boolean isCapturing() {
202             return true;
203         }
204
205         @Override
206         public boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
207             log.debug("capture should not be called in state {}", this);
208             return false;
209         }
210
211         @Override
212         public boolean captureToInstall(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
213                 final String targetFollower) {
214             log.debug("captureToInstall should not be called in state {}", this);
215             return false;
216         }
217
218         @Override
219         public boolean captureWithForcedTrim(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
220             log.debug("captureWithForcedTrim should not be called in state {}", this);
221             return false;
222         }
223
224         @Override
225         public void apply(final ApplySnapshot snapshot) {
226             log.debug("apply should not be called in state {}", this);
227         }
228
229         @Override
230         public void persist(final Snapshot.State state, final Optional<OutputStream> installSnapshotStream,
231                 final long totalMemory) {
232             log.debug("persist should not be called in state {}", this);
233         }
234
235         @Override
236         public void commit(final long sequenceNumber, final long timeStamp) {
237             log.debug("commit should not be called in state {}", this);
238         }
239
240         @Override
241         public void rollback() {
242             log.debug("rollback should not be called in state {}", this);
243         }
244
245         @Override
246         public long trimLog(final long desiredTrimIndex) {
247             log.debug("trimLog should not be called in state {}", this);
248             return -1;
249         }
250
251         protected long doTrimLog(final long desiredTrimIndex) {
252             //  we would want to keep the lastApplied as its used while capturing snapshots
253             long lastApplied = context.getLastApplied();
254             long tempMin = Math.min(desiredTrimIndex, lastApplied > -1 ? lastApplied - 1 : -1);
255
256             if (log.isTraceEnabled()) {
257                 log.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, tempMin: {}",
258                         persistenceId(), desiredTrimIndex, lastApplied, tempMin);
259             }
260
261             if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) {
262                 log.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin,
263                         context.getTermInformation().getCurrentTerm());
264
265                 //use the term of the temp-min, since we check for isPresent, entry will not be null
266                 ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
267                 context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
268                 context.getReplicatedLog().snapshotCommit();
269                 return tempMin;
270             }
271
272             final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
273             if (tempMin > currentBehavior.getReplicatedToAllIndex()) {
274                 // It's possible a follower was lagging and an install snapshot advanced its match index past
275                 // the current replicatedToAllIndex. Since the follower is now caught up we should advance the
276                 // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely
277                 // due to a previous snapshot triggered by the memory threshold exceeded, in that case we
278                 // trim the log to the last applied index even if previous entries weren't replicated to all followers.
279                 currentBehavior.setReplicatedToAllIndex(tempMin);
280             }
281             return -1;
282         }
283     }
284
285     private class Idle extends AbstractSnapshotState {
286
287         @Override
288         public boolean isCapturing() {
289             return false;
290         }
291
292         @SuppressWarnings("checkstyle:IllegalCatch")
293         private boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
294                 final String targetFollower, final boolean mandatoryTrim) {
295             captureSnapshot = newCaptureSnapshot(lastLogEntry, replicatedToAllIndex, mandatoryTrim);
296
297             OutputStream installSnapshotStream = null;
298             if (targetFollower != null) {
299                 installSnapshotStream = context.getFileBackedOutputStreamFactory().newInstance();
300                 log.info("{}: Initiating snapshot capture {} to install on {}",
301                         persistenceId(), captureSnapshot, targetFollower);
302             } else {
303                 log.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
304             }
305
306             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
307
308             log.debug("{}: lastSequenceNumber prior to capture: {}", persistenceId(), lastSequenceNumber);
309
310             SnapshotManager.this.currentState = CREATING;
311
312             try {
313                 createSnapshotProcedure.accept(Optional.ofNullable(installSnapshotStream));
314             } catch (Exception e) {
315                 SnapshotManager.this.currentState = IDLE;
316                 log.error("Error creating snapshot", e);
317                 return false;
318             }
319
320             return true;
321         }
322
323         @Override
324         public boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
325             return capture(lastLogEntry, replicatedToAllIndex, null, false);
326         }
327
328         @Override
329         public boolean captureToInstall(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
330                 final String targetFollower) {
331             return capture(lastLogEntry, replicatedToAllIndex, targetFollower, false);
332         }
333
334         @Override
335         public boolean captureWithForcedTrim(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
336             return capture(lastLogEntry, replicatedToAllIndex, null, true);
337         }
338
339         @Override
340         public void apply(final ApplySnapshot toApply) {
341             SnapshotManager.this.applySnapshot = toApply;
342
343             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
344
345             log.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber);
346
347             context.getPersistenceProvider().saveSnapshot(toApply.getSnapshot());
348
349             SnapshotManager.this.currentState = PERSISTING;
350         }
351
352         @Override
353         public String toString() {
354             return "Idle";
355         }
356
357         @Override
358         public long trimLog(final long desiredTrimIndex) {
359             return doTrimLog(desiredTrimIndex);
360         }
361     }
362
363     private class Creating extends AbstractSnapshotState {
364
365         @Override
366         public void persist(final Snapshot.State snapshotState, final Optional<OutputStream> installSnapshotStream,
367                 final long totalMemory) {
368             // create a snapshot object from the state provided and save it
369             // when snapshot is saved async, SaveSnapshotSuccess is raised.
370
371             Snapshot snapshot = Snapshot.create(snapshotState,
372                     captureSnapshot.getUnAppliedEntries(),
373                     captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
374                     captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm(),
375                     context.getTermInformation().getCurrentTerm(),
376                     context.getTermInformation().getVotedFor(), context.getPeerServerInfo(true));
377
378             context.getPersistenceProvider().saveSnapshot(snapshot);
379
380             log.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot);
381
382             final ConfigParams config = context.getConfigParams();
383             final long absoluteThreshold = config.getSnapshotDataThreshold();
384             final long dataThreshold = absoluteThreshold != 0 ? absoluteThreshold * ConfigParams.MEGABYTE
385                     : totalMemory * config.getSnapshotDataThresholdPercentage() / 100;
386
387             final boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold;
388             final boolean logSizeExceededSnapshotBatchCount =
389                     context.getReplicatedLog().size() >= config.getSnapshotBatchCount();
390
391             final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
392             if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount || captureSnapshot.isMandatoryTrim()) {
393                 if (log.isDebugEnabled()) {
394                     if (dataSizeThresholdExceeded) {
395                         log.debug("{}: log data size {} exceeds the memory threshold {} - doing snapshotPreCommit "
396                                 + "with index {}", context.getId(), context.getReplicatedLog().dataSize(),
397                                 dataThreshold, captureSnapshot.getLastAppliedIndex());
398                     } else if (logSizeExceededSnapshotBatchCount) {
399                         log.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with "
400                                 + "index {}", context.getId(), context.getReplicatedLog().size(),
401                                 config.getSnapshotBatchCount(), captureSnapshot.getLastAppliedIndex());
402                     } else {
403                         log.debug("{}: user triggered or root overwrite snapshot encountered, trimming log up to "
404                                 + "last applied index {}", context.getId(), captureSnapshot.getLastAppliedIndex());
405                     }
406                 }
407
408                 // We either exceeded the memory threshold or the log size exceeded the snapshot batch
409                 // count so, to keep the log memory footprint in check, clear the log based on lastApplied.
410                 // This could/should only happen if one of the followers is down as normally we keep
411                 // removing from the log as entries are replicated to all.
412                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
413                         captureSnapshot.getLastAppliedTerm());
414
415                 // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
416                 // install snapshot to a follower.
417                 if (captureSnapshot.getReplicatedToAllIndex() >= 0) {
418                     currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
419                 }
420
421             } else if (captureSnapshot.getReplicatedToAllIndex() != -1) {
422                 // clear the log based on replicatedToAllIndex
423                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
424                         captureSnapshot.getReplicatedToAllTerm());
425
426                 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
427             } else {
428                 // The replicatedToAllIndex was not found in the log
429                 // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
430                 // In this scenario we may need to save the snapshot to the akka persistence
431                 // snapshot for recovery but we do not need to do the replicated log trimming.
432                 context.getReplicatedLog().snapshotPreCommit(context.getReplicatedLog().getSnapshotIndex(),
433                         context.getReplicatedLog().getSnapshotTerm());
434             }
435
436             log.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} and term: {}",
437                     context.getId(), context.getReplicatedLog().getSnapshotIndex(),
438                     context.getReplicatedLog().getSnapshotTerm());
439
440             if (installSnapshotStream.isPresent()) {
441                 if (context.getId().equals(currentBehavior.getLeaderId())) {
442                     try {
443                         ByteSource snapshotBytes = ((FileBackedOutputStream)installSnapshotStream.get()).asByteSource();
444                         currentBehavior.handleMessage(context.getActor(),
445                                 new SendInstallSnapshot(snapshot, snapshotBytes));
446                     } catch (IOException e) {
447                         log.error("{}: Snapshot install failed due to an unrecoverable streaming error",
448                                 context.getId(), e);
449                     }
450                 } else {
451                     ((FileBackedOutputStream)installSnapshotStream.get()).cleanup();
452                 }
453             }
454
455             captureSnapshot = null;
456             SnapshotManager.this.currentState = PERSISTING;
457         }
458
459         @Override
460         public String toString() {
461             return "Creating";
462         }
463
464     }
465
466     private class Persisting extends AbstractSnapshotState {
467
468         @Override
469         @SuppressWarnings("checkstyle:IllegalCatch")
470         public void commit(final long sequenceNumber, final long timeStamp) {
471             log.debug("{}: Snapshot success -  sequence number: {}", persistenceId(), sequenceNumber);
472
473             if (applySnapshot != null) {
474                 try {
475                     Snapshot snapshot = applySnapshot.getSnapshot();
476
477                     //clears the followers log, sets the snapshot index to ensure adjusted-index works
478                     context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context));
479                     context.setLastApplied(snapshot.getLastAppliedIndex());
480                     context.setCommitIndex(snapshot.getLastAppliedIndex());
481                     context.getTermInformation().update(snapshot.getElectionTerm(), snapshot.getElectionVotedFor());
482
483                     if (snapshot.getServerConfiguration() != null) {
484                         context.updatePeerIds(snapshot.getServerConfiguration());
485                     }
486
487                     if (!(snapshot.getState() instanceof EmptyState)) {
488                         snapshotCohort.applySnapshot(snapshot.getState());
489                     }
490
491                     applySnapshot.getCallback().onSuccess();
492                 } catch (Exception e) {
493                     log.error("{}: Error applying snapshot", context.getId(), e);
494                 }
495             } else {
496                 context.getReplicatedLog().snapshotCommit();
497             }
498
499             context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(),
500                     timeStamp - 1, 0L, 0L));
501
502             context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
503
504             snapshotComplete();
505         }
506
507         @Override
508         public void rollback() {
509             // Nothing to rollback if we're applying a snapshot from the leader.
510             if (applySnapshot == null) {
511                 context.getReplicatedLog().snapshotRollback();
512
513                 log.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle."
514                         + "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
515                         context.getReplicatedLog().getSnapshotIndex(),
516                         context.getReplicatedLog().getSnapshotTerm(),
517                         context.getReplicatedLog().size());
518             } else {
519                 applySnapshot.getCallback().onFailure();
520             }
521
522             snapshotComplete();
523         }
524
525         private void snapshotComplete() {
526             lastSequenceNumber = -1;
527             applySnapshot = null;
528             SnapshotManager.this.currentState = IDLE;
529
530             context.getActor().tell(SnapshotComplete.INSTANCE, context.getActor());
531         }
532
533         @Override
534         public String toString() {
535             return "Persisting";
536         }
537
538     }
539
540     private interface TermInformationReader {
541         long getIndex();
542
543         long getTerm();
544     }
545
546     static class LastAppliedTermInformationReader implements TermInformationReader {
547         private long index;
548         private long term;
549
550         LastAppliedTermInformationReader init(final ReplicatedLog log, final long originalIndex,
551                 final ReplicatedLogEntry lastLogEntry, final boolean hasFollowers) {
552             ReplicatedLogEntry entry = log.get(originalIndex);
553             this.index = -1L;
554             this.term = -1L;
555             if (!hasFollowers) {
556                 if (lastLogEntry != null) {
557                     // since we have persisted the last-log-entry to persistent journal before the capture,
558                     // we would want to snapshot from this entry.
559                     index = lastLogEntry.getIndex();
560                     term = lastLogEntry.getTerm();
561                 }
562             } else if (entry != null) {
563                 index = entry.getIndex();
564                 term = entry.getTerm();
565             } else if (log.getSnapshotIndex() > -1) {
566                 index = log.getSnapshotIndex();
567                 term = log.getSnapshotTerm();
568             }
569             return this;
570         }
571
572         @Override
573         public long getIndex() {
574             return this.index;
575         }
576
577         @Override
578         public long getTerm() {
579             return this.term;
580         }
581     }
582
583     private static class ReplicatedToAllTermInformationReader implements TermInformationReader {
584         private long index;
585         private long term;
586
587         ReplicatedToAllTermInformationReader init(final ReplicatedLog log, final long originalIndex) {
588             ReplicatedLogEntry entry = log.get(originalIndex);
589             this.index = -1L;
590             this.term = -1L;
591
592             if (entry != null) {
593                 index = entry.getIndex();
594                 term = entry.getTerm();
595             }
596
597             return this;
598         }
599
600         @Override
601         public long getIndex() {
602             return this.index;
603         }
604
605         @Override
606         public long getTerm() {
607             return this.term;
608         }
609     }
610 }