Lost commit index when a snapshot is captured
[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 void apply(final 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, final 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     @SuppressWarnings("checkstyle:hiddenField")
121     void setCreateSnapshotConsumer(final Consumer<Optional<OutputStream>> createSnapshotProcedure) {
122         this.createSnapshotProcedure = createSnapshotProcedure;
123     }
124
125     void setSnapshotCohort(final RaftActorSnapshotCohort snapshotCohort) {
126         this.snapshotCohort = snapshotCohort;
127     }
128
129     public Snapshot.@NonNull State convertSnapshot(final 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(final ReplicatedLogEntry lastLogEntry, final 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         final long lastLogEntryIndex;
174         final long lastLogEntryTerm;
175         if (lastLogEntry == null) {
176             // When we don't have journal present, for example two captureSnapshots executed right after another with no
177             // new journal we still want to preserve the index and term in the snapshot.
178             lastAppliedIndex = lastLogEntryIndex = context.getReplicatedLog().getSnapshotIndex();
179             lastAppliedTerm = lastLogEntryTerm = context.getReplicatedLog().getSnapshotTerm();
180
181             log.debug("{}: Capturing Snapshot : lastLogEntry is null. Using snapshot values lastAppliedIndex {} and "
182                     + "lastAppliedTerm {} instead.", persistenceId(), lastAppliedIndex, lastAppliedTerm);
183         } else {
184             lastLogEntryIndex = lastLogEntry.getIndex();
185             lastLogEntryTerm = lastLogEntry.getTerm();
186         }
187
188         return new CaptureSnapshot(lastLogEntryIndex, lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm,
189                 newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries);
190     }
191
192     private class AbstractSnapshotState implements SnapshotState {
193
194         @Override
195         public boolean isCapturing() {
196             return true;
197         }
198
199         @Override
200         public boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
201             log.debug("capture should not be called in state {}", this);
202             return false;
203         }
204
205         @Override
206         public boolean captureToInstall(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
207                 final String targetFollower) {
208             log.debug("captureToInstall should not be called in state {}", this);
209             return false;
210         }
211
212         @Override
213         public void apply(final ApplySnapshot snapshot) {
214             log.debug("apply should not be called in state {}", this);
215         }
216
217         @Override
218         public void persist(final Snapshot.State state, final Optional<OutputStream> installSnapshotStream,
219                 final long totalMemory) {
220             log.debug("persist should not be called in state {}", this);
221         }
222
223         @Override
224         public void commit(final long sequenceNumber, final long timeStamp) {
225             log.debug("commit should not be called in state {}", this);
226         }
227
228         @Override
229         public void rollback() {
230             log.debug("rollback should not be called in state {}", this);
231         }
232
233         @Override
234         public long trimLog(final long desiredTrimIndex) {
235             log.debug("trimLog should not be called in state {}", this);
236             return -1;
237         }
238
239         protected long doTrimLog(final long desiredTrimIndex) {
240             //  we would want to keep the lastApplied as its used while capturing snapshots
241             long lastApplied = context.getLastApplied();
242             long tempMin = Math.min(desiredTrimIndex, lastApplied > -1 ? lastApplied - 1 : -1);
243
244             if (log.isTraceEnabled()) {
245                 log.trace("{}: performSnapshotWithoutCapture: desiredTrimIndex: {}, lastApplied: {}, tempMin: {}",
246                         persistenceId(), desiredTrimIndex, lastApplied, tempMin);
247             }
248
249             if (tempMin > -1 && context.getReplicatedLog().isPresent(tempMin)) {
250                 log.debug("{}: fakeSnapshot purging log to {} for term {}", persistenceId(), tempMin,
251                         context.getTermInformation().getCurrentTerm());
252
253                 //use the term of the temp-min, since we check for isPresent, entry will not be null
254                 ReplicatedLogEntry entry = context.getReplicatedLog().get(tempMin);
255                 context.getReplicatedLog().snapshotPreCommit(tempMin, entry.getTerm());
256                 context.getReplicatedLog().snapshotCommit();
257                 return tempMin;
258             }
259
260             final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
261             if (tempMin > currentBehavior.getReplicatedToAllIndex()) {
262                 // It's possible a follower was lagging and an install snapshot advanced its match index past
263                 // the current replicatedToAllIndex. Since the follower is now caught up we should advance the
264                 // replicatedToAllIndex (to tempMin). The fact that tempMin wasn't found in the log is likely
265                 // due to a previous snapshot triggered by the memory threshold exceeded, in that case we
266                 // trim the log to the last applied index even if previous entries weren't replicated to all followers.
267                 currentBehavior.setReplicatedToAllIndex(tempMin);
268             }
269             return -1;
270         }
271     }
272
273     private class Idle extends AbstractSnapshotState {
274
275         @Override
276         public boolean isCapturing() {
277             return false;
278         }
279
280         @SuppressWarnings("checkstyle:IllegalCatch")
281         private boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
282                 final String targetFollower) {
283             captureSnapshot = newCaptureSnapshot(lastLogEntry, replicatedToAllIndex);
284
285             OutputStream installSnapshotStream = null;
286             if (targetFollower != null) {
287                 installSnapshotStream = context.getFileBackedOutputStreamFactory().newInstance();
288                 log.info("{}: Initiating snapshot capture {} to install on {}",
289                         persistenceId(), captureSnapshot, targetFollower);
290             } else {
291                 log.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
292             }
293
294             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
295
296             log.debug("{}: lastSequenceNumber prior to capture: {}", persistenceId(), lastSequenceNumber);
297
298             SnapshotManager.this.currentState = CREATING;
299
300             try {
301                 createSnapshotProcedure.accept(Optional.ofNullable(installSnapshotStream));
302             } catch (Exception e) {
303                 SnapshotManager.this.currentState = IDLE;
304                 log.error("Error creating snapshot", e);
305                 return false;
306             }
307
308             return true;
309         }
310
311         @Override
312         public boolean capture(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex) {
313             return capture(lastLogEntry, replicatedToAllIndex, null);
314         }
315
316         @Override
317         public boolean captureToInstall(final ReplicatedLogEntry lastLogEntry, final long replicatedToAllIndex,
318                 final String targetFollower) {
319             return capture(lastLogEntry, replicatedToAllIndex, targetFollower);
320         }
321
322         @Override
323         public void apply(final ApplySnapshot toApply) {
324             SnapshotManager.this.applySnapshot = toApply;
325
326             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
327
328             log.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber);
329
330             context.getPersistenceProvider().saveSnapshot(toApply.getSnapshot());
331
332             SnapshotManager.this.currentState = PERSISTING;
333         }
334
335         @Override
336         public String toString() {
337             return "Idle";
338         }
339
340         @Override
341         public long trimLog(final long desiredTrimIndex) {
342             return doTrimLog(desiredTrimIndex);
343         }
344     }
345
346     private class Creating extends AbstractSnapshotState {
347
348         @Override
349         public void persist(final Snapshot.State snapshotState, final Optional<OutputStream> installSnapshotStream,
350                 final long totalMemory) {
351             // create a snapshot object from the state provided and save it
352             // when snapshot is saved async, SaveSnapshotSuccess is raised.
353
354             Snapshot snapshot = Snapshot.create(snapshotState,
355                     captureSnapshot.getUnAppliedEntries(),
356                     captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
357                     captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm(),
358                     context.getTermInformation().getCurrentTerm(),
359                     context.getTermInformation().getVotedFor(), context.getPeerServerInfo(true));
360
361             context.getPersistenceProvider().saveSnapshot(snapshot);
362
363             log.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot);
364
365             long dataThreshold = totalMemory * context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
366             boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold;
367
368             boolean logSizeExceededSnapshotBatchCount =
369                     context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount();
370
371             final RaftActorBehavior currentBehavior = context.getCurrentBehavior();
372             if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount) {
373                 if (log.isDebugEnabled()) {
374                     if (dataSizeThresholdExceeded) {
375                         log.debug("{}: log data size {} exceeds the memory threshold {} - doing snapshotPreCommit "
376                                 + "with index {}", context.getId(), context.getReplicatedLog().dataSize(),
377                                 dataThreshold, captureSnapshot.getLastAppliedIndex());
378                     } else {
379                         log.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with "
380                                 + "index {}", context.getId(), context.getReplicatedLog().size(),
381                                 context.getConfigParams().getSnapshotBatchCount(),
382                                 captureSnapshot.getLastAppliedIndex());
383                     }
384                 }
385
386                 // We either exceeded the memory threshold or the log size exceeded the snapshot batch
387                 // count so, to keep the log memory footprint in check, clear the log based on lastApplied.
388                 // This could/should only happen if one of the followers is down as normally we keep
389                 // removing from the log as entries are replicated to all.
390                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
391                         captureSnapshot.getLastAppliedTerm());
392
393                 // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
394                 // install snapshot to a follower.
395                 if (captureSnapshot.getReplicatedToAllIndex() >= 0) {
396                     currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
397                 }
398
399             } else if (captureSnapshot.getReplicatedToAllIndex() != -1) {
400                 // clear the log based on replicatedToAllIndex
401                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
402                         captureSnapshot.getReplicatedToAllTerm());
403
404                 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
405             } else {
406                 // The replicatedToAllIndex was not found in the log
407                 // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
408                 // In this scenario we may need to save the snapshot to the akka persistence
409                 // snapshot for recovery but we do not need to do the replicated log trimming.
410                 context.getReplicatedLog().snapshotPreCommit(context.getReplicatedLog().getSnapshotIndex(),
411                         context.getReplicatedLog().getSnapshotTerm());
412             }
413
414             log.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} and term: {}",
415                     context.getId(), context.getReplicatedLog().getSnapshotIndex(),
416                     context.getReplicatedLog().getSnapshotTerm());
417
418             if (installSnapshotStream.isPresent()) {
419                 if (context.getId().equals(currentBehavior.getLeaderId())) {
420                     try {
421                         ByteSource snapshotBytes = ((FileBackedOutputStream)installSnapshotStream.get()).asByteSource();
422                         currentBehavior.handleMessage(context.getActor(),
423                                 new SendInstallSnapshot(snapshot, snapshotBytes));
424                     } catch (IOException e) {
425                         log.error("{}: Snapshot install failed due to an unrecoverable streaming error",
426                                 context.getId(), e);
427                     }
428                 } else {
429                     ((FileBackedOutputStream)installSnapshotStream.get()).cleanup();
430                 }
431             }
432
433             captureSnapshot = null;
434             SnapshotManager.this.currentState = PERSISTING;
435         }
436
437         @Override
438         public String toString() {
439             return "Creating";
440         }
441
442     }
443
444     private class Persisting extends AbstractSnapshotState {
445
446         @Override
447         @SuppressWarnings("checkstyle:IllegalCatch")
448         public void commit(final long sequenceNumber, final long timeStamp) {
449             log.debug("{}: Snapshot success -  sequence number: {}", persistenceId(), sequenceNumber);
450
451             if (applySnapshot != null) {
452                 try {
453                     Snapshot snapshot = applySnapshot.getSnapshot();
454
455                     //clears the followers log, sets the snapshot index to ensure adjusted-index works
456                     context.setReplicatedLog(ReplicatedLogImpl.newInstance(snapshot, context));
457                     context.setLastApplied(snapshot.getLastAppliedIndex());
458                     context.setCommitIndex(snapshot.getLastAppliedIndex());
459                     context.getTermInformation().update(snapshot.getElectionTerm(), snapshot.getElectionVotedFor());
460
461                     if (snapshot.getServerConfiguration() != null) {
462                         context.updatePeerIds(snapshot.getServerConfiguration());
463                     }
464
465                     if (!(snapshot.getState() instanceof EmptyState)) {
466                         snapshotCohort.applySnapshot(snapshot.getState());
467                     }
468
469                     applySnapshot.getCallback().onSuccess();
470                 } catch (Exception e) {
471                     log.error("{}: Error applying snapshot", context.getId(), e);
472                 }
473             } else {
474                 context.getReplicatedLog().snapshotCommit();
475             }
476
477             context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(scala.Long.MaxValue(),
478                     timeStamp - 1, 0L, 0L));
479
480             context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
481
482             snapshotComplete();
483         }
484
485         @Override
486         public void rollback() {
487             // Nothing to rollback if we're applying a snapshot from the leader.
488             if (applySnapshot == null) {
489                 context.getReplicatedLog().snapshotRollback();
490
491                 log.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle."
492                         + "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
493                         context.getReplicatedLog().getSnapshotIndex(),
494                         context.getReplicatedLog().getSnapshotTerm(),
495                         context.getReplicatedLog().size());
496             } else {
497                 applySnapshot.getCallback().onFailure();
498             }
499
500             snapshotComplete();
501         }
502
503         private void snapshotComplete() {
504             lastSequenceNumber = -1;
505             applySnapshot = null;
506             SnapshotManager.this.currentState = IDLE;
507
508             context.getActor().tell(SnapshotComplete.INSTANCE, context.getActor());
509         }
510
511         @Override
512         public String toString() {
513             return "Persisting";
514         }
515
516     }
517
518     private interface TermInformationReader {
519         long getIndex();
520
521         long getTerm();
522     }
523
524     static class LastAppliedTermInformationReader implements TermInformationReader {
525         private long index;
526         private long term;
527
528         LastAppliedTermInformationReader init(final ReplicatedLog log, final long originalIndex,
529                 final ReplicatedLogEntry lastLogEntry, final boolean hasFollowers) {
530             ReplicatedLogEntry entry = log.get(originalIndex);
531             this.index = -1L;
532             this.term = -1L;
533             if (!hasFollowers) {
534                 if (lastLogEntry != null) {
535                     // since we have persisted the last-log-entry to persistent journal before the capture,
536                     // we would want to snapshot from this entry.
537                     index = lastLogEntry.getIndex();
538                     term = lastLogEntry.getTerm();
539                 }
540             } else if (entry != null) {
541                 index = entry.getIndex();
542                 term = entry.getTerm();
543             } else if (log.getSnapshotIndex() > -1) {
544                 index = log.getSnapshotIndex();
545                 term = log.getSnapshotTerm();
546             }
547             return this;
548         }
549
550         @Override
551         public long getIndex() {
552             return this.index;
553         }
554
555         @Override
556         public long getTerm() {
557             return this.term;
558         }
559     }
560
561     private static class ReplicatedToAllTermInformationReader implements TermInformationReader {
562         private long index;
563         private long term;
564
565         ReplicatedToAllTermInformationReader init(final ReplicatedLog log, final long originalIndex) {
566             ReplicatedLogEntry entry = log.get(originalIndex);
567             this.index = -1L;
568             this.term = -1L;
569
570             if (entry != null) {
571                 index = entry.getIndex();
572                 term = entry.getTerm();
573             }
574
575             return this;
576         }
577
578         @Override
579         public long getIndex() {
580             return this.index;
581         }
582
583         @Override
584         public long getTerm() {
585             return this.term;
586         }
587     }
588 }