0b7808391162327188cc871c69caf75422b88c4c
[controller.git] / opendaylight / md-sal / sal-akka-segmented-journal / src / main / java / org / opendaylight / controller / akka / segjournal / SegmentedJournalActor.java
1 /*
2  * Copyright (c) 2019 Pantheon Technologies, s.r.o. 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 package org.opendaylight.controller.akka.segjournal;
9
10 import static com.google.common.base.Verify.verify;
11 import static com.google.common.base.Verify.verifyNotNull;
12 import static java.util.Objects.requireNonNull;
13
14 import akka.actor.AbstractActor;
15 import akka.actor.ActorRef;
16 import akka.actor.Props;
17 import akka.japi.pf.ReceiveBuilder;
18 import akka.persistence.AtomicWrite;
19 import akka.persistence.PersistentRepr;
20 import com.codahale.metrics.Histogram;
21 import com.codahale.metrics.Meter;
22 import com.codahale.metrics.MetricRegistry;
23 import com.codahale.metrics.Timer;
24 import com.google.common.base.MoreObjects;
25 import com.google.common.base.Stopwatch;
26 import io.atomix.storage.journal.Indexed;
27 import io.atomix.storage.journal.JournalSerdes;
28 import io.atomix.storage.journal.SegmentedByteBufJournal;
29 import io.atomix.storage.journal.SegmentedJournal;
30 import io.atomix.storage.journal.StorageLevel;
31 import java.io.File;
32 import java.util.ArrayDeque;
33 import java.util.ArrayList;
34 import java.util.List;
35 import java.util.Optional;
36 import java.util.concurrent.TimeUnit;
37 import java.util.function.Consumer;
38 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
39 import org.opendaylight.controller.cluster.reporting.MetricsReporter;
40 import org.opendaylight.controller.raft.journal.FromByteBufMapper;
41 import org.opendaylight.controller.raft.journal.ToByteBufMapper;
42 import org.slf4j.Logger;
43 import org.slf4j.LoggerFactory;
44 import scala.concurrent.Future;
45 import scala.concurrent.Promise;
46
47 /**
48  * This actor handles a single PersistentActor's journal. The journal is split into two {@link SegmentedJournal}s:
49  * <ul>
50  *     <li>A memory-mapped data journal, containing actual data entries</li>
51  *     <li>A simple file journal, containing sequence numbers of last deleted entry</li>
52  * </ul>
53  *
54  * <p>
55  * This is a conscious design decision to minimize the amount of data that is being stored in the data journal while
56  * speeding up normal operations. Since the SegmentedJournal is an append-only linear log and Akka requires the ability
57  * to delete persistence entries, we need ability to mark a subset of a SegmentedJournal as deleted. While we could
58  * treat such delete requests as normal events, this leads to a mismatch between SegmentedJournal indices (as exposed by
59  * {@link Indexed}) and Akka sequence numbers -- requiring us to potentially perform costly deserialization to find the
60  * index corresponding to a particular sequence number, or maintain moderately-complex logic and data structures to
61  * perform that mapping in sub-linear time complexity.
62  *
63  * <p>
64  * Split-file approach allows us to treat sequence numbers and indices as equivalent, without maintaining any explicit
65  * mapping information. The only additional information we need to maintain is the last deleted sequence number.
66  */
67 abstract sealed class SegmentedJournalActor extends AbstractActor {
68     abstract static sealed class AsyncMessage<T> {
69         final Promise<T> promise = Promise.apply();
70     }
71
72     private static final class ReadHighestSequenceNr extends AsyncMessage<Long> {
73         private final long fromSequenceNr;
74
75         ReadHighestSequenceNr(final long fromSequenceNr) {
76             this.fromSequenceNr = fromSequenceNr;
77         }
78
79         @Override
80         public String toString() {
81             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr).toString();
82         }
83     }
84
85     static final class ReplayMessages extends AsyncMessage<Void> {
86         private final long fromSequenceNr;
87         final long toSequenceNr;
88         final long max;
89         final Consumer<PersistentRepr> replayCallback;
90
91         ReplayMessages(final long fromSequenceNr,
92                 final long toSequenceNr, final long max, final Consumer<PersistentRepr> replayCallback) {
93             this.fromSequenceNr = fromSequenceNr;
94             this.toSequenceNr = toSequenceNr;
95             this.max = max;
96             this.replayCallback = requireNonNull(replayCallback);
97         }
98
99         @Override
100         public String toString() {
101             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr)
102                     .add("toSequenceNr", toSequenceNr).add("max", max).toString();
103         }
104     }
105
106     static final class WriteMessages {
107         private final List<AtomicWrite> requests = new ArrayList<>();
108         private final List<Promise<Optional<Exception>>> results = new ArrayList<>();
109
110         Future<Optional<Exception>> add(final AtomicWrite write) {
111             final var promise = Promise.<Optional<Exception>>apply();
112             requests.add(write);
113             results.add(promise);
114             return promise.future();
115         }
116
117         int size() {
118             return requests.size();
119         }
120
121         AtomicWrite getRequest(final int index) {
122             return requests.get(index);
123         }
124
125         void setFailure(final int index, final Exception cause) {
126             results.get(index).success(Optional.of(cause));
127         }
128
129         void setSuccess(final int index) {
130             results.get(index).success(Optional.empty());
131         }
132
133         @Override
134         public String toString() {
135             return MoreObjects.toStringHelper(this).add("requests", requests).toString();
136         }
137     }
138
139     private static final class DeleteMessagesTo extends AsyncMessage<Void> {
140         final long toSequenceNr;
141
142         DeleteMessagesTo(final long toSequenceNr) {
143             this.toSequenceNr = toSequenceNr;
144         }
145
146         @Override
147         public String toString() {
148             return MoreObjects.toStringHelper(this).add("toSequenceNr", toSequenceNr).toString();
149         }
150     }
151
152     // responses == null on success, Exception on failure
153     record WrittenMessages(WriteMessages message, List<Object> responses, long writtenBytes) {
154         WrittenMessages {
155             verify(responses.size() == message.size(), "Mismatched %s and %s", message, responses);
156             verify(writtenBytes >= 0, "Unexpected length %s", writtenBytes);
157         }
158
159         private void complete() {
160             for (int i = 0, size = responses.size(); i < size; ++i) {
161                 if (responses.get(i) instanceof Exception ex) {
162                     message.setFailure(i, ex);
163                 } else {
164                     message.setSuccess(i);
165                 }
166             }
167         }
168     }
169
170     /**
171      * A {@link SegmentedJournalActor} which delays issuing a flush operation until a watermark is reached or when the
172      * queue is empty.
173      *
174      * <p>
175      * The problem we are addressing is that there is a queue sitting in from of the actor, which we have no direct
176      * access to. Since a flush involves committing data to durable storage, that operation can easily end up dominating
177      * workloads.
178      *
179      * <p>
180      * We solve this by having an additional queue in which we track which messages were written and trigger a flush
181      * only when the number of bytes we have written exceeds specified limit. The other part is that each time this
182      * queue becomes non-empty, we send a dedicated message to self. This acts as a actor queue probe -- when we receive
183      * it, we know we have processed all messages that were in the queue when we first delayed the write.
184      *
185      * <p>
186      * The combination of these mechanisms ensure we use a minimal delay while also ensuring we take advantage of
187      * batching opportunities.
188      */
189     private static final class Delayed extends SegmentedJournalActor {
190         private static final class Flush extends AsyncMessage<Void> {
191             final long batch;
192
193             Flush(final long batch) {
194                 this.batch = batch;
195             }
196         }
197
198         private record UnflushedWrite(WrittenMessages message, Stopwatch start, long count) {
199             UnflushedWrite {
200                 requireNonNull(message);
201                 requireNonNull(start);
202             }
203         }
204
205         private final ArrayDeque<UnflushedWrite> unflushedWrites = new ArrayDeque<>();
206         private final Stopwatch unflushedDuration = Stopwatch.createUnstarted();
207         private final long maxUnflushedBytes;
208
209         private long batch = 0;
210         private long unflushedBytes = 0;
211
212         Delayed(final String persistenceId, final File directory, final StorageLevel storage,
213                 final int maxEntrySize, final int maxSegmentSize, final int maxUnflushedBytes) {
214             super(persistenceId, directory, storage, maxEntrySize, maxSegmentSize);
215             this.maxUnflushedBytes = maxUnflushedBytes;
216         }
217
218         @Override
219         ReceiveBuilder addMessages(final ReceiveBuilder builder) {
220             return super.addMessages(builder).match(Flush.class, this::handleFlush);
221         }
222
223         private void handleFlush(final Flush message) {
224             if (message.batch == batch) {
225                 flushWrites();
226             } else {
227                 LOG.debug("{}: batch {} not flushed by {}", persistenceId(), batch, message.batch);
228             }
229         }
230
231         @Override
232         void onWrittenMessages(final WrittenMessages message, final Stopwatch started, final long count) {
233             boolean first = unflushedWrites.isEmpty();
234             if (first) {
235                 unflushedDuration.start();
236             }
237             unflushedWrites.addLast(new UnflushedWrite(message, started, count));
238             unflushedBytes = unflushedBytes + message.writtenBytes;
239             if (unflushedBytes >= maxUnflushedBytes) {
240                 LOG.debug("{}: reached {} unflushed journal bytes", persistenceId(), unflushedBytes);
241                 flushWrites();
242             } else if (first) {
243                 LOG.debug("{}: deferring journal flush", persistenceId());
244                 self().tell(new Flush(++batch), ActorRef.noSender());
245             }
246         }
247
248         @Override
249         void flushWrites() {
250             final var unsyncedSize = unflushedWrites.size();
251             if (unsyncedSize == 0) {
252                 // Nothing to flush
253                 return;
254             }
255
256             LOG.debug("{}: flushing {} journal writes after {}", persistenceId(), unsyncedSize,
257                 unflushedDuration.stop());
258             flushJournal(unflushedBytes, unsyncedSize);
259
260             final var sw = Stopwatch.createStarted();
261             unflushedWrites.forEach(write -> completeWriteMessages(write.message, write.start, write.count));
262             unflushedWrites.clear();
263             unflushedBytes = 0;
264             unflushedDuration.reset();
265             LOG.debug("{}: completed {} flushed journal writes in {}", persistenceId(), unsyncedSize, sw);
266         }
267     }
268
269     private static final class Immediate extends SegmentedJournalActor {
270         Immediate(final String persistenceId, final File directory, final StorageLevel storage,
271                 final int maxEntrySize, final int maxSegmentSize) {
272             super(persistenceId, directory, storage, maxEntrySize, maxSegmentSize);
273         }
274
275         @Override
276         void onWrittenMessages(final WrittenMessages message, final Stopwatch started, final long count) {
277             flushJournal(message.writtenBytes, 1);
278             completeWriteMessages(message, started, count);
279         }
280
281         @Override
282         void flushWrites() {
283             // No-op
284         }
285     }
286
287     private static final Logger LOG = LoggerFactory.getLogger(SegmentedJournalActor.class);
288     private static final int DELETE_SEGMENT_SIZE = 64 * 1024;
289     private static final FromByteBufMapper<Long> READ_MAPPER;
290     private static final ToByteBufMapper<Long> WRITE_MAPPER;
291
292     static {
293         final var namespace = JournalSerdes.builder()
294             .register(LongEntrySerdes.LONG_ENTRY_SERDES, Long.class)
295             .build();
296
297         READ_MAPPER = namespace.toReadMapper();
298         WRITE_MAPPER = namespace.toWriteMapper();
299     }
300
301     private final String persistenceId;
302     private final StorageLevel storage;
303     private final int maxSegmentSize;
304     private final int maxEntrySize;
305     private final File directory;
306
307     // Tracks the time it took us to write a batch of messages
308     private Timer batchWriteTime;
309     // Tracks the number of individual messages written
310     private Meter messageWriteCount;
311     // Tracks the size distribution of messages
312     private Histogram messageSize;
313     // Tracks the number of messages completed for each flush
314     private Histogram flushMessages;
315     // Tracks the number of bytes completed for each flush
316     private Histogram flushBytes;
317     // Tracks the duration of flush operations
318     private Timer flushTime;
319
320     private DataJournal dataJournal;
321     private SegmentedJournal<Long> deleteJournal;
322     private long lastDelete;
323
324     private SegmentedJournalActor(final String persistenceId, final File directory, final StorageLevel storage,
325             final int maxEntrySize, final int maxSegmentSize) {
326         this.persistenceId = requireNonNull(persistenceId);
327         this.directory = requireNonNull(directory);
328         this.storage = requireNonNull(storage);
329         this.maxEntrySize = maxEntrySize;
330         this.maxSegmentSize = maxSegmentSize;
331     }
332
333     static Props props(final String persistenceId, final File directory, final StorageLevel storage,
334             final int maxEntrySize, final int maxSegmentSize, final int maxUnflushedBytes) {
335         final var pid = requireNonNull(persistenceId);
336         return maxUnflushedBytes > 0
337             ? Props.create(Delayed.class, pid, directory, storage, maxEntrySize, maxSegmentSize, maxUnflushedBytes)
338             : Props.create(Immediate.class, pid, directory, storage, maxEntrySize, maxSegmentSize);
339     }
340
341     final String persistenceId() {
342         return persistenceId;
343     }
344
345     final void flushJournal(final long bytes, final int messages) {
346         final var sw = Stopwatch.createStarted();
347         dataJournal.flush();
348         LOG.debug("{}: journal flush completed in {}", persistenceId, sw.stop());
349         flushBytes.update(bytes);
350         flushMessages.update(messages);
351         flushTime.update(sw.elapsed(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
352     }
353
354     @Override
355     public Receive createReceive() {
356         return addMessages(receiveBuilder())
357             .matchAny(this::handleUnknown)
358             .build();
359     }
360
361     ReceiveBuilder addMessages(final ReceiveBuilder builder) {
362         return builder
363             .match(DeleteMessagesTo.class, this::handleDeleteMessagesTo)
364             .match(ReadHighestSequenceNr.class, this::handleReadHighestSequenceNr)
365             .match(ReplayMessages.class, this::handleReplayMessages)
366             .match(WriteMessages.class, this::handleWriteMessages);
367     }
368
369     @Override
370     public void preStart() throws Exception {
371         LOG.debug("{}: actor starting", persistenceId);
372         super.preStart();
373
374         final var registry = MetricsReporter.getInstance(MeteringBehavior.DOMAIN).getMetricsRegistry();
375         final var actorName = self().path().parent().toStringWithoutAddress() + '/' + directory.getName();
376
377         batchWriteTime = registry.timer(MetricRegistry.name(actorName, "batchWriteTime"));
378         messageWriteCount = registry.meter(MetricRegistry.name(actorName, "messageWriteCount"));
379         messageSize = registry.histogram(MetricRegistry.name(actorName, "messageSize"));
380         flushBytes = registry.histogram(MetricRegistry.name(actorName, "flushBytes"));
381         flushMessages = registry.histogram(MetricRegistry.name(actorName, "flushMessages"));
382         flushTime = registry.timer(MetricRegistry.name(actorName, "flushTime"));
383     }
384
385     @Override
386     public void postStop() throws Exception {
387         LOG.debug("{}: actor stopping", persistenceId);
388         if (dataJournal != null) {
389             dataJournal.close();
390             LOG.debug("{}: data journal closed", persistenceId);
391             dataJournal = null;
392         }
393         if (deleteJournal != null) {
394             deleteJournal.close();
395             LOG.debug("{}: delete journal closed", persistenceId);
396             deleteJournal = null;
397         }
398         LOG.debug("{}: actor stopped", persistenceId);
399         super.postStop();
400     }
401
402     static AsyncMessage<Void> deleteMessagesTo(final long toSequenceNr) {
403         return new DeleteMessagesTo(toSequenceNr);
404     }
405
406     static AsyncMessage<Long> readHighestSequenceNr(final long fromSequenceNr) {
407         return new ReadHighestSequenceNr(fromSequenceNr);
408     }
409
410     static AsyncMessage<Void> replayMessages(final long fromSequenceNr, final long toSequenceNr, final long max,
411             final Consumer<PersistentRepr> replayCallback) {
412         return new ReplayMessages(fromSequenceNr, toSequenceNr, max, replayCallback);
413     }
414
415     private void handleDeleteMessagesTo(final DeleteMessagesTo message) {
416         ensureOpen();
417
418         LOG.debug("{}: delete messages {}", persistenceId, message);
419         flushWrites();
420
421         final long to = Long.min(dataJournal.lastWrittenSequenceNr(), message.toSequenceNr);
422         LOG.debug("{}: adjusted delete to {}", persistenceId, to);
423
424         if (lastDelete < to) {
425             LOG.debug("{}: deleting entries up to {}", persistenceId, to);
426
427             lastDelete = to;
428             final var deleteWriter = deleteJournal.writer();
429             final var entry = deleteWriter.append(lastDelete);
430             deleteWriter.commit(entry.index());
431             dataJournal.deleteTo(lastDelete);
432
433             LOG.debug("{}: compaction started", persistenceId);
434             dataJournal.compactTo(lastDelete);
435             deleteJournal.compact(entry.index());
436             LOG.debug("{}: compaction finished", persistenceId);
437         } else {
438             LOG.debug("{}: entries up to {} already deleted", persistenceId, lastDelete);
439         }
440
441         message.promise.success(null);
442     }
443
444     private void handleReadHighestSequenceNr(final ReadHighestSequenceNr message) {
445         LOG.debug("{}: looking for highest sequence on {}", persistenceId, message);
446         final Long sequence;
447         if (directory.isDirectory()) {
448             ensureOpen();
449             flushWrites();
450             sequence = dataJournal.lastWrittenSequenceNr();
451         } else {
452             sequence = 0L;
453         }
454
455         LOG.debug("{}: highest sequence is {}", message, sequence);
456         message.promise.success(sequence);
457     }
458
459     private void handleReplayMessages(final ReplayMessages message) {
460         LOG.debug("{}: replaying messages {}", persistenceId, message);
461         ensureOpen();
462         flushWrites();
463
464         final long from = Long.max(lastDelete + 1, message.fromSequenceNr);
465         LOG.debug("{}: adjusted fromSequenceNr to {}", persistenceId, from);
466
467         dataJournal.handleReplayMessages(message, from);
468     }
469
470     private void handleWriteMessages(final WriteMessages message) {
471         ensureOpen();
472
473         final var started = Stopwatch.createStarted();
474         final long start = dataJournal.lastWrittenSequenceNr();
475         final var writtenMessages = dataJournal.handleWriteMessages(message);
476
477         onWrittenMessages(writtenMessages, started, dataJournal.lastWrittenSequenceNr() - start);
478     }
479
480     final void completeWriteMessages(final WrittenMessages message, final Stopwatch started, final long count) {
481         batchWriteTime.update(started.stop().elapsed(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
482         messageWriteCount.mark(count);
483         // log message after statistics are updated
484         LOG.debug("{}: write of {} bytes completed in {}", persistenceId, message.writtenBytes, started);
485         message.complete();
486     }
487
488     /**
489      * Handle a check of written messages.
490      *
491      * @param message Messages which were written
492      * @param started Stopwatch started when the write started
493      * @param count number of writes
494      */
495     abstract void onWrittenMessages(WrittenMessages message, Stopwatch started, long count);
496
497     private void handleUnknown(final Object message) {
498         LOG.error("{}: Received unknown message {}", persistenceId, message);
499     }
500
501     private void ensureOpen() {
502         if (dataJournal != null) {
503             verifyNotNull(deleteJournal);
504             return;
505         }
506
507         final var sw = Stopwatch.createStarted();
508         deleteJournal = new SegmentedJournal<>(SegmentedByteBufJournal.builder()
509             .withDirectory(directory)
510             .withName("delete")
511             .withMaxSegmentSize(DELETE_SEGMENT_SIZE)
512             .build(), READ_MAPPER, WRITE_MAPPER);
513         final var lastDeleteRecovered = deleteJournal.openReader(deleteJournal.lastIndex())
514             .tryNext((index, value, length) -> value);
515         lastDelete = lastDeleteRecovered == null ? 0 : lastDeleteRecovered;
516
517         dataJournal = new DataJournalV0(persistenceId, messageSize, context().system(), storage, directory,
518             maxEntrySize, maxSegmentSize);
519         dataJournal.deleteTo(lastDelete);
520         LOG.debug("{}: journal open in {} with last index {}, deleted to {}", persistenceId, sw,
521             dataJournal.lastWrittenSequenceNr(), lastDelete);
522     }
523
524     abstract void flushWrites();
525
526 }