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