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