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