Do not leak Kryo from atomix.storage
[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.verifyNotNull;
11 import static java.util.Objects.requireNonNull;
12
13 import akka.actor.AbstractActor;
14 import akka.actor.Props;
15 import akka.persistence.AtomicWrite;
16 import akka.persistence.PersistentRepr;
17 import com.codahale.metrics.Histogram;
18 import com.codahale.metrics.Meter;
19 import com.codahale.metrics.MetricRegistry;
20 import com.codahale.metrics.Timer;
21 import com.google.common.base.MoreObjects;
22 import com.google.common.base.Stopwatch;
23 import io.atomix.storage.journal.Indexed;
24 import io.atomix.storage.journal.JournalSerdes;
25 import io.atomix.storage.journal.SegmentedJournal;
26 import io.atomix.storage.journal.SegmentedJournalWriter;
27 import io.atomix.storage.journal.StorageLevel;
28 import java.io.File;
29 import java.util.ArrayList;
30 import java.util.List;
31 import java.util.Optional;
32 import java.util.concurrent.TimeUnit;
33 import java.util.function.Consumer;
34 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
35 import org.opendaylight.controller.cluster.reporting.MetricsReporter;
36 import org.slf4j.Logger;
37 import org.slf4j.LoggerFactory;
38 import scala.concurrent.Future;
39 import scala.concurrent.Promise;
40
41 /**
42  * This actor handles a single PersistentActor's journal. The journal is split into two {@link SegmentedJournal}s:
43  * <ul>
44  *     <li>A memory-mapped data journal, containing actual data entries</li>
45  *     <li>A simple file journal, containing sequence numbers of last deleted entry</li>
46  * </ul>
47  *
48  * <p>
49  * This is a conscious design decision to minimize the amount of data that is being stored in the data journal while
50  * speeding up normal operations. Since the SegmentedJournal is an append-only linear log and Akka requires the ability
51  * to delete persistence entries, we need ability to mark a subset of a SegmentedJournal as deleted. While we could
52  * treat such delete requests as normal events, this leads to a mismatch between SegmentedJournal indices (as exposed by
53  * {@link Indexed}) and Akka sequence numbers -- requiring us to potentially perform costly deserialization to find the
54  * index corresponding to a particular sequence number, or maintain moderately-complex logic and data structures to
55  * perform that mapping in sub-linear time complexity.
56  *
57  * <p>
58  * Split-file approach allows us to treat sequence numbers and indices as equivalent, without maintaining any explicit
59  * mapping information. The only additional information we need to maintain is the last deleted sequence number.
60  *
61  * @author Robert Varga
62  */
63 final class SegmentedJournalActor extends AbstractActor {
64     abstract static class AsyncMessage<T> {
65         final Promise<T> promise = Promise.apply();
66     }
67
68     private static final class ReadHighestSequenceNr extends AsyncMessage<Long> {
69         private final long fromSequenceNr;
70
71         ReadHighestSequenceNr(final long fromSequenceNr) {
72             this.fromSequenceNr = fromSequenceNr;
73         }
74
75         @Override
76         public String toString() {
77             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr).toString();
78         }
79     }
80
81     static final class ReplayMessages extends AsyncMessage<Void> {
82         private final long fromSequenceNr;
83         final long toSequenceNr;
84         final long max;
85         final Consumer<PersistentRepr> replayCallback;
86
87         ReplayMessages(final long fromSequenceNr,
88                 final long toSequenceNr, final long max, final Consumer<PersistentRepr> replayCallback) {
89             this.fromSequenceNr = fromSequenceNr;
90             this.toSequenceNr = toSequenceNr;
91             this.max = max;
92             this.replayCallback = requireNonNull(replayCallback);
93         }
94
95         @Override
96         public String toString() {
97             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr)
98                     .add("toSequenceNr", toSequenceNr).add("max", max).toString();
99         }
100     }
101
102     static final class WriteMessages {
103         private final List<AtomicWrite> requests = new ArrayList<>();
104         private final List<Promise<Optional<Exception>>> results = new ArrayList<>();
105
106         Future<Optional<Exception>> add(final AtomicWrite write) {
107             final Promise<Optional<Exception>> promise = Promise.apply();
108             requests.add(write);
109             results.add(promise);
110             return promise.future();
111         }
112
113         int size() {
114             return requests.size();
115         }
116
117         AtomicWrite getRequest(final int index) {
118             return requests.get(index);
119         }
120
121         void setFailure(final int index, final Exception cause) {
122             results.get(index).success(Optional.of(cause));
123
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     private static final Logger LOG = LoggerFactory.getLogger(SegmentedJournalActor.class);
150     private static final JournalSerdes DELETE_NAMESPACE = JournalSerdes.builder()
151         .register(new LongSerdes(), Long.class)
152         .build();
153     private static final int DELETE_SEGMENT_SIZE = 64 * 1024;
154
155     private final String persistenceId;
156     private final StorageLevel storage;
157     private final int maxSegmentSize;
158     private final int maxEntrySize;
159     private final File directory;
160
161     // Tracks the time it took us to write a batch of messages
162     private Timer batchWriteTime;
163     // Tracks the number of individual messages written
164     private Meter messageWriteCount;
165     // Tracks the size distribution of messages
166     private Histogram messageSize;
167
168     private DataJournal dataJournal;
169     private SegmentedJournal<Long> deleteJournal;
170     private long lastDelete;
171
172     SegmentedJournalActor(final String persistenceId, final File directory, final StorageLevel storage,
173             final int maxEntrySize, final int maxSegmentSize) {
174         this.persistenceId = requireNonNull(persistenceId);
175         this.directory = requireNonNull(directory);
176         this.storage = requireNonNull(storage);
177         this.maxEntrySize = maxEntrySize;
178         this.maxSegmentSize = maxSegmentSize;
179     }
180
181     static Props props(final String persistenceId, final File directory, final StorageLevel storage,
182             final int maxEntrySize, final int maxSegmentSize) {
183         return Props.create(SegmentedJournalActor.class, requireNonNull(persistenceId), directory, storage,
184             maxEntrySize, maxSegmentSize);
185     }
186
187     @Override
188     public Receive createReceive() {
189         return receiveBuilder()
190                 .match(DeleteMessagesTo.class, this::handleDeleteMessagesTo)
191                 .match(ReadHighestSequenceNr.class, this::handleReadHighestSequenceNr)
192                 .match(ReplayMessages.class, this::handleReplayMessages)
193                 .match(WriteMessages.class, this::handleWriteMessages)
194                 .matchAny(this::handleUnknown)
195                 .build();
196     }
197
198     @Override
199     public void preStart() throws Exception {
200         LOG.debug("{}: actor starting", persistenceId);
201         super.preStart();
202
203         final MetricRegistry registry = MetricsReporter.getInstance(MeteringBehavior.DOMAIN).getMetricsRegistry();
204         final String actorName = self().path().parent().toStringWithoutAddress() + '/' + directory.getName();
205
206         batchWriteTime = registry.timer(MetricRegistry.name(actorName, "batchWriteTime"));
207         messageWriteCount = registry.meter(MetricRegistry.name(actorName, "messageWriteCount"));
208         messageSize = registry.histogram(MetricRegistry.name(actorName, "messageSize"));
209     }
210
211     @Override
212     public void postStop() throws Exception {
213         LOG.debug("{}: actor stopping", persistenceId);
214         if (dataJournal != null) {
215             dataJournal.close();
216             LOG.debug("{}: data journal closed", persistenceId);
217             dataJournal = null;
218         }
219         if (deleteJournal != null) {
220             deleteJournal.close();
221             LOG.debug("{}: delete journal closed", persistenceId);
222             deleteJournal = null;
223         }
224         LOG.debug("{}: actor stopped", persistenceId);
225         super.postStop();
226     }
227
228     static AsyncMessage<Void> deleteMessagesTo(final long toSequenceNr) {
229         return new DeleteMessagesTo(toSequenceNr);
230     }
231
232     static AsyncMessage<Long> readHighestSequenceNr(final long fromSequenceNr) {
233         return new ReadHighestSequenceNr(fromSequenceNr);
234     }
235
236     static AsyncMessage<Void> replayMessages(final long fromSequenceNr, final long toSequenceNr, final long max,
237             final Consumer<PersistentRepr> replayCallback) {
238         return new ReplayMessages(fromSequenceNr, toSequenceNr, max, replayCallback);
239     }
240
241     private void handleDeleteMessagesTo(final DeleteMessagesTo message) {
242         ensureOpen();
243
244         LOG.debug("{}: delete messages {}", persistenceId, message);
245         final long to = Long.min(dataJournal.lastWrittenSequenceNr(), message.toSequenceNr);
246         LOG.debug("{}: adjusted delete to {}", persistenceId, to);
247
248         if (lastDelete < to) {
249             LOG.debug("{}: deleting entries up to {}", persistenceId, to);
250
251             lastDelete = to;
252             final SegmentedJournalWriter<Long> deleteWriter = deleteJournal.writer();
253             final Indexed<Long> entry = deleteWriter.append(lastDelete);
254             deleteWriter.commit(entry.index());
255             dataJournal.deleteTo(lastDelete);
256
257             LOG.debug("{}: compaction started", persistenceId);
258             dataJournal.compactTo(lastDelete);
259             deleteJournal.compact(entry.index());
260             LOG.debug("{}: compaction finished", persistenceId);
261         } else {
262             LOG.debug("{}: entries up to {} already deleted", persistenceId, lastDelete);
263         }
264
265         message.promise.success(null);
266     }
267
268     private void handleReadHighestSequenceNr(final ReadHighestSequenceNr message) {
269         LOG.debug("{}: looking for highest sequence on {}", persistenceId, message);
270         final Long sequence;
271         if (directory.isDirectory()) {
272             ensureOpen();
273             sequence = dataJournal.lastWrittenSequenceNr();
274         } else {
275             sequence = 0L;
276         }
277
278         LOG.debug("{}: highest sequence is {}", message, sequence);
279         message.promise.success(sequence);
280     }
281
282     private void handleReplayMessages(final ReplayMessages message) {
283         LOG.debug("{}: replaying messages {}", persistenceId, message);
284         ensureOpen();
285
286         final long from = Long.max(lastDelete + 1, message.fromSequenceNr);
287         LOG.debug("{}: adjusted fromSequenceNr to {}", persistenceId, from);
288
289         dataJournal.handleReplayMessages(message, from);
290     }
291
292     private void handleWriteMessages(final WriteMessages message) {
293         ensureOpen();
294
295         final Stopwatch sw = Stopwatch.createStarted();
296         final long start = dataJournal.lastWrittenSequenceNr();
297         final long bytes = dataJournal.handleWriteMessages(message);
298         sw.stop();
299
300         batchWriteTime.update(sw.elapsed(TimeUnit.NANOSECONDS), TimeUnit.NANOSECONDS);
301         messageWriteCount.mark(dataJournal.lastWrittenSequenceNr() - start);
302
303         // log message after statistics are updated
304         LOG.debug("{}: write of {} bytes completed in {}", persistenceId, bytes, sw);
305     }
306
307     private void handleUnknown(final Object message) {
308         LOG.error("{}: Received unknown message {}", persistenceId, message);
309     }
310
311     private void ensureOpen() {
312         if (dataJournal != null) {
313             verifyNotNull(deleteJournal);
314             return;
315         }
316
317         final Stopwatch sw = Stopwatch.createStarted();
318         deleteJournal = SegmentedJournal.<Long>builder().withDirectory(directory).withName("delete")
319                 .withNamespace(DELETE_NAMESPACE).withMaxSegmentSize(DELETE_SEGMENT_SIZE).build();
320         final Indexed<Long> lastEntry = deleteJournal.writer().getLastEntry();
321         lastDelete = lastEntry == null ? 0 : lastEntry.entry();
322
323         dataJournal = new DataJournalV0(persistenceId, messageSize, context().system(), storage, directory,
324             maxEntrySize, maxSegmentSize);
325         dataJournal.deleteTo(lastDelete);
326         LOG.debug("{}: journal open in {} with last index {}, deleted to {}", persistenceId, sw,
327             dataJournal.lastWrittenSequenceNr(), lastDelete);
328     }
329 }