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