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