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