Bug 2461 : Adding capture snapshot JMX operation.
[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             long lastLogEntryIndex = lastAppliedIndex;
215             long lastLogEntryTerm = lastAppliedTerm;
216             if(lastLogEntry != null) {
217                 lastLogEntryIndex = lastLogEntry.getIndex();
218                 lastLogEntryTerm = lastLogEntry.getTerm();
219             } else {
220                 LOG.warn("Capturing Snapshot : lastLogEntry is null. Using lastAppliedIndex {} and lastAppliedTerm {} instead.",
221                     lastAppliedIndex, lastAppliedTerm);
222             }
223
224             captureSnapshot = new CaptureSnapshot(lastLogEntryIndex,
225                 lastLogEntryTerm, lastAppliedIndex, lastAppliedTerm,
226                     newReplicatedToAllIndex, newReplicatedToAllTerm, unAppliedEntries, targetFollower != null);
227
228             if(captureSnapshot.isInstallSnapshotInitiated()) {
229                 LOG.info("{}: Initiating snapshot capture {} to install on {}",
230                         persistenceId(), captureSnapshot, targetFollower);
231             } else {
232                 LOG.info("{}: Initiating snapshot capture {}", persistenceId(), captureSnapshot);
233             }
234
235             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
236
237             LOG.debug("lastSequenceNumber prior to capture: {}", lastSequenceNumber);
238
239             SnapshotManager.this.currentState = CREATING;
240
241             try {
242                 createSnapshotProcedure.apply(null);
243             } catch (Exception e) {
244                 SnapshotManager.this.currentState = IDLE;
245                 LOG.error("Error creating snapshot", e);
246                 return false;
247             }
248
249             return true;
250         }
251
252         @Override
253         public boolean capture(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex) {
254             return capture(lastLogEntry, replicatedToAllIndex, null);
255         }
256
257         @Override
258         public boolean captureToInstall(ReplicatedLogEntry lastLogEntry, long replicatedToAllIndex, String targetFollower) {
259             return capture(lastLogEntry, replicatedToAllIndex, targetFollower);
260         }
261
262         @Override
263         public void apply(Snapshot snapshot) {
264             applySnapshot = snapshot;
265
266             lastSequenceNumber = context.getPersistenceProvider().getLastSequenceNumber();
267
268             LOG.debug("lastSequenceNumber prior to persisting applied snapshot: {}", lastSequenceNumber);
269
270             context.getPersistenceProvider().saveSnapshot(snapshot);
271
272             SnapshotManager.this.currentState = PERSISTING;
273         }
274
275         @Override
276         public String toString() {
277             return "Idle";
278         }
279
280         @Override
281         public long trimLog(long desiredTrimIndex, RaftActorBehavior currentBehavior) {
282             return doTrimLog(desiredTrimIndex, currentBehavior);
283         }
284     }
285
286     private class Creating extends AbstractSnapshotState {
287
288         @Override
289         public void persist(byte[] snapshotBytes, RaftActorBehavior currentBehavior, long totalMemory) {
290             // create a snapshot object from the state provided and save it
291             // when snapshot is saved async, SaveSnapshotSuccess is raised.
292
293             Snapshot snapshot = Snapshot.create(snapshotBytes,
294                     captureSnapshot.getUnAppliedEntries(),
295                     captureSnapshot.getLastIndex(), captureSnapshot.getLastTerm(),
296                     captureSnapshot.getLastAppliedIndex(), captureSnapshot.getLastAppliedTerm());
297
298             context.getPersistenceProvider().saveSnapshot(snapshot);
299
300             LOG.info("{}: Persisting of snapshot done: {}", persistenceId(), snapshot.getLogMessage());
301
302             long dataThreshold = totalMemory *
303                     context.getConfigParams().getSnapshotDataThresholdPercentage() / 100;
304             boolean dataSizeThresholdExceeded = context.getReplicatedLog().dataSize() > dataThreshold;
305
306             boolean logSizeExceededSnapshotBatchCount =
307                     context.getReplicatedLog().size() >= context.getConfigParams().getSnapshotBatchCount();
308 LOG.debug("Log size: {}, getSnapshotBatchCount: {}",context.getReplicatedLog().size(),context.getConfigParams().getSnapshotBatchCount());
309             if (dataSizeThresholdExceeded || logSizeExceededSnapshotBatchCount) {
310                 if(LOG.isDebugEnabled()) {
311                     if(dataSizeThresholdExceeded) {
312                         LOG.debug("{}: log data size {} exceeds the memory threshold {} - doing snapshotPreCommit with index {}",
313                                 context.getId(), context.getReplicatedLog().dataSize(), dataThreshold,
314                                 captureSnapshot.getLastAppliedIndex());
315                     } else {
316                         LOG.debug("{}: log size {} exceeds the snapshot batch count {} - doing snapshotPreCommit with index {}",
317                                 context.getId(), context.getReplicatedLog().size(),
318                                 context.getConfigParams().getSnapshotBatchCount(), captureSnapshot.getLastAppliedIndex());
319                     }
320                 }
321
322                 // We either exceeded the memory threshold or the log size exceeded the snapshot batch
323                 // count so, to keep the log memory footprint in check, clear the log based on lastApplied.
324                 // This could/should only happen if one of the followers is down as normally we keep
325                 // removing from the log as entries are replicated to all.
326                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getLastAppliedIndex(),
327                         captureSnapshot.getLastAppliedTerm());
328
329                 // Don't reset replicatedToAllIndex to -1 as this may prevent us from trimming the log after an
330                 // install snapshot to a follower.
331                 if(captureSnapshot.getReplicatedToAllIndex() >= 0) {
332                     currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
333                 }
334
335             } else if(captureSnapshot.getReplicatedToAllIndex() != -1){
336                 // clear the log based on replicatedToAllIndex
337                 context.getReplicatedLog().snapshotPreCommit(captureSnapshot.getReplicatedToAllIndex(),
338                         captureSnapshot.getReplicatedToAllTerm());
339
340                 currentBehavior.setReplicatedToAllIndex(captureSnapshot.getReplicatedToAllIndex());
341             } else {
342                 // The replicatedToAllIndex was not found in the log
343                 // This means that replicatedToAllIndex never moved beyond -1 or that it is already in the snapshot.
344                 // In this scenario we may need to save the snapshot to the akka persistence
345                 // snapshot for recovery but we do not need to do the replicated log trimming.
346                 context.getReplicatedLog().snapshotPreCommit(context.getReplicatedLog().getSnapshotIndex(),
347                         context.getReplicatedLog().getSnapshotTerm());
348             }
349
350             LOG.info("{}: Removed in-memory snapshotted entries, adjusted snaphsotIndex: {} " +
351                     "and term: {}", context.getId(), context.getReplicatedLog().getSnapshotIndex(),
352                     context.getReplicatedLog().getSnapshotTerm());
353
354             if (context.getId().equals(currentBehavior.getLeaderId())
355                     && captureSnapshot.isInstallSnapshotInitiated()) {
356                 // this would be call straight to the leader and won't initiate in serialization
357                 currentBehavior.handleMessage(context.getActor(), new SendInstallSnapshot(snapshot));
358             }
359
360             captureSnapshot = null;
361             SnapshotManager.this.currentState = PERSISTING;
362         }
363
364         @Override
365         public String toString() {
366             return "Creating";
367         }
368
369     }
370
371     private class Persisting extends AbstractSnapshotState {
372
373         @Override
374         public void commit(long sequenceNumber, RaftActorBehavior currentBehavior) {
375             LOG.debug("Snapshot success sequence number:", sequenceNumber);
376
377             if(applySnapshot != null) {
378                 try {
379                     applySnapshotProcedure.apply(applySnapshot.getState());
380
381                     //clears the followers log, sets the snapshot index to ensure adjusted-index works
382                     context.setReplicatedLog(ReplicatedLogImpl.newInstance(applySnapshot, context, currentBehavior));
383                     context.setLastApplied(applySnapshot.getLastAppliedIndex());
384                     context.setCommitIndex(applySnapshot.getLastAppliedIndex());
385                 } catch (Exception e) {
386                     LOG.error("Error applying snapshot", e);
387                 }
388             } else {
389                 context.getReplicatedLog().snapshotCommit();
390             }
391
392             context.getPersistenceProvider().deleteSnapshots(new SnapshotSelectionCriteria(
393                     sequenceNumber - context.getConfigParams().getSnapshotBatchCount(), 43200000));
394
395             context.getPersistenceProvider().deleteMessages(lastSequenceNumber);
396
397             lastSequenceNumber = -1;
398             applySnapshot = null;
399             SnapshotManager.this.currentState = IDLE;
400         }
401
402         @Override
403         public void rollback() {
404             // Nothing to rollback if we're applying a snapshot from the leader.
405             if(applySnapshot == null) {
406                 context.getReplicatedLog().snapshotRollback();
407
408                 LOG.info("{}: Replicated Log rolled back. Snapshot will be attempted in the next cycle." +
409                         "snapshotIndex:{}, snapshotTerm:{}, log-size:{}", persistenceId(),
410                         context.getReplicatedLog().getSnapshotIndex(),
411                         context.getReplicatedLog().getSnapshotTerm(),
412                         context.getReplicatedLog().size());
413             }
414
415             lastSequenceNumber = -1;
416             applySnapshot = null;
417             SnapshotManager.this.currentState = IDLE;
418         }
419
420         @Override
421         public String toString() {
422             return "Persisting";
423         }
424
425     }
426
427     private static interface TermInformationReader {
428         long getIndex();
429         long getTerm();
430     }
431
432     static class LastAppliedTermInformationReader implements TermInformationReader{
433         private long index;
434         private long term;
435
436         public LastAppliedTermInformationReader init(ReplicatedLog log, long originalIndex,
437                                          ReplicatedLogEntry lastLogEntry, boolean hasFollowers){
438             ReplicatedLogEntry entry = log.get(originalIndex);
439             this.index = -1L;
440             this.term = -1L;
441             if (!hasFollowers) {
442                 if(lastLogEntry != null) {
443                     // since we have persisted the last-log-entry to persistent journal before the capture,
444                     // we would want to snapshot from this entry.
445                     index = lastLogEntry.getIndex();
446                     term = lastLogEntry.getTerm();
447                 }
448             } else if (entry != null) {
449                 index = entry.getIndex();
450                 term = entry.getTerm();
451             } else if(log.getSnapshotIndex() > -1){
452                 index = log.getSnapshotIndex();
453                 term = log.getSnapshotTerm();
454             }
455             return this;
456         }
457
458         @Override
459         public long getIndex(){
460             return this.index;
461         }
462
463         @Override
464         public long getTerm(){
465             return this.term;
466         }
467     }
468
469     private static class ReplicatedToAllTermInformationReader implements TermInformationReader{
470         private long index;
471         private long term;
472
473         ReplicatedToAllTermInformationReader init(ReplicatedLog log, long originalIndex){
474             ReplicatedLogEntry entry = log.get(originalIndex);
475             this.index = -1L;
476             this.term = -1L;
477
478             if (entry != null) {
479                 index = entry.getIndex();
480                 term = entry.getTerm();
481             }
482
483             return this;
484         }
485
486         @Override
487         public long getIndex(){
488             return this.index;
489         }
490
491         @Override
492         public long getTerm(){
493             return this.term;
494         }
495     }
496 }