1739b25cab023c19f72be61675813977faf6e90a
[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.Props;
16 import akka.persistence.AtomicWrite;
17 import akka.persistence.PersistentRepr;
18 import com.codahale.metrics.Histogram;
19 import com.codahale.metrics.Meter;
20 import com.codahale.metrics.MetricRegistry;
21 import com.codahale.metrics.Timer;
22 import com.google.common.base.MoreObjects;
23 import io.atomix.storage.StorageLevel;
24 import io.atomix.storage.journal.Indexed;
25 import io.atomix.storage.journal.SegmentedJournal;
26 import io.atomix.storage.journal.SegmentedJournalReader;
27 import io.atomix.storage.journal.SegmentedJournalWriter;
28 import io.atomix.utils.serializer.Namespace;
29 import java.io.File;
30 import java.io.Serializable;
31 import java.util.ArrayList;
32 import java.util.List;
33 import java.util.Optional;
34 import java.util.concurrent.TimeUnit;
35 import java.util.function.Consumer;
36 import org.opendaylight.controller.akka.segjournal.DataJournalEntry.FromPersistence;
37 import org.opendaylight.controller.akka.segjournal.DataJournalEntry.ToPersistence;
38 import org.opendaylight.controller.cluster.common.actor.MeteringBehavior;
39 import org.opendaylight.controller.cluster.reporting.MetricsReporter;
40 import org.slf4j.Logger;
41 import org.slf4j.LoggerFactory;
42 import scala.collection.Iterator;
43 import scala.collection.SeqLike;
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  * @author Robert Varga
68  */
69 final class SegmentedJournalActor extends AbstractActor {
70     abstract static class AsyncMessage<T> {
71         final Promise<T> promise = Promise.apply();
72     }
73
74     private static final class ReadHighestSequenceNr extends AsyncMessage<Long> {
75         private final long fromSequenceNr;
76
77         ReadHighestSequenceNr(final long fromSequenceNr) {
78             this.fromSequenceNr = fromSequenceNr;
79         }
80
81         @Override
82         public String toString() {
83             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr).toString();
84         }
85     }
86
87     private static final class ReplayMessages extends AsyncMessage<Void> {
88         private final long fromSequenceNr;
89         private final long toSequenceNr;
90         private final long max;
91         private final Consumer<PersistentRepr> replayCallback;
92
93         ReplayMessages(final long fromSequenceNr,
94                 final long toSequenceNr, final long max, final Consumer<PersistentRepr> replayCallback) {
95             this.fromSequenceNr = fromSequenceNr;
96             this.toSequenceNr = toSequenceNr;
97             this.max = max;
98             this.replayCallback = requireNonNull(replayCallback);
99         }
100
101         @Override
102         public String toString() {
103             return MoreObjects.toStringHelper(this).add("fromSequenceNr", fromSequenceNr)
104                     .add("toSequenceNr", toSequenceNr).add("max", max).toString();
105         }
106     }
107
108     static final class WriteMessages {
109         private final List<AtomicWrite> requests = new ArrayList<>();
110         private final List<Promise<Optional<Exception>>> results = new ArrayList<>();
111
112         Future<Optional<Exception>> add(final AtomicWrite write) {
113             final Promise<Optional<Exception>> promise = Promise.apply();
114             requests.add(write);
115             results.add(promise);
116             return promise.future();
117         }
118
119         @Override
120         public String toString() {
121             return MoreObjects.toStringHelper(this).add("requests", requests).toString();
122         }
123     }
124
125     private static final class DeleteMessagesTo extends AsyncMessage<Void> {
126         final long toSequenceNr;
127
128         DeleteMessagesTo(final long toSequenceNr) {
129             this.toSequenceNr = toSequenceNr;
130         }
131
132         @Override
133         public String toString() {
134             return MoreObjects.toStringHelper(this).add("toSequenceNr", toSequenceNr).toString();
135         }
136     }
137
138     private static final Logger LOG = LoggerFactory.getLogger(SegmentedJournalActor.class);
139     private static final Namespace DELETE_NAMESPACE = Namespace.builder().register(Long.class).build();
140     private static final int DELETE_SEGMENT_SIZE = 64 * 1024;
141
142     private final String persistenceId;
143     private final StorageLevel storage;
144     private final int maxSegmentSize;
145     private final int maxEntrySize;
146     private final File directory;
147
148     // Tracks the time it took us to write a batch of messages
149     private Timer batchWriteTime;
150     // Tracks the number of individual messages written
151     private Meter messageWriteCount;
152     // Tracks the size distribution of messages
153     private Histogram messageSize;
154
155     private SegmentedJournal<DataJournalEntry> dataJournal;
156     private SegmentedJournal<Long> deleteJournal;
157     private long lastDelete;
158
159     // Tracks largest message size we have observed either during recovery or during write
160     private int largestObservedSize;
161
162     SegmentedJournalActor(final String persistenceId, final File directory, final StorageLevel storage,
163             final int maxEntrySize, final int maxSegmentSize) {
164         this.persistenceId = requireNonNull(persistenceId);
165         this.directory = requireNonNull(directory);
166         this.storage = requireNonNull(storage);
167         this.maxEntrySize = maxEntrySize;
168         this.maxSegmentSize = maxSegmentSize;
169     }
170
171     static Props props(final String persistenceId, final File directory, final StorageLevel storage,
172             final int maxEntrySize, final int maxSegmentSize) {
173         return Props.create(SegmentedJournalActor.class, requireNonNull(persistenceId), directory, storage,
174             maxEntrySize, maxSegmentSize);
175     }
176
177     @Override
178     public Receive createReceive() {
179         return receiveBuilder()
180                 .match(DeleteMessagesTo.class, this::handleDeleteMessagesTo)
181                 .match(ReadHighestSequenceNr.class, this::handleReadHighestSequenceNr)
182                 .match(ReplayMessages.class, this::handleReplayMessages)
183                 .match(WriteMessages.class, this::handleWriteMessages)
184                 .matchAny(this::handleUnknown)
185                 .build();
186     }
187
188     @Override
189     public void preStart() throws Exception {
190         LOG.debug("{}: actor starting", persistenceId);
191         super.preStart();
192
193         final MetricRegistry registry = MetricsReporter.getInstance(MeteringBehavior.DOMAIN).getMetricsRegistry();
194         final String actorName = self().path().parent().toStringWithoutAddress() + '/' + directory.getName();
195
196         batchWriteTime = registry.timer(MetricRegistry.name(actorName, "batchWriteTime"));
197         messageWriteCount = registry.meter(MetricRegistry.name(actorName, "messageWriteCount"));
198         messageSize = registry.histogram(MetricRegistry.name(actorName, "messageSize"));
199     }
200
201     @Override
202     public void postStop() throws Exception {
203         LOG.debug("{}: actor stopping", persistenceId);
204         if (dataJournal != null) {
205             dataJournal.close();
206             LOG.debug("{}: data journal closed", persistenceId);
207             dataJournal = null;
208         }
209         if (deleteJournal != null) {
210             deleteJournal.close();
211             LOG.debug("{}: delete journal closed", persistenceId);
212             deleteJournal = null;
213         }
214         LOG.debug("{}: actor stopped", persistenceId);
215         super.postStop();
216     }
217
218     static AsyncMessage<Void> deleteMessagesTo(final long toSequenceNr) {
219         return new DeleteMessagesTo(toSequenceNr);
220     }
221
222     static AsyncMessage<Long> readHighestSequenceNr(final long fromSequenceNr) {
223         return new ReadHighestSequenceNr(fromSequenceNr);
224     }
225
226     static AsyncMessage<Void> replayMessages(final long fromSequenceNr, final long toSequenceNr, final long max,
227             final Consumer<PersistentRepr> replayCallback) {
228         return new ReplayMessages(fromSequenceNr, toSequenceNr, max, replayCallback);
229     }
230
231     private void handleDeleteMessagesTo(final DeleteMessagesTo message) {
232         ensureOpen();
233
234         LOG.debug("{}: delete messages {}", persistenceId, message);
235         final long to = Long.min(dataJournal.writer().getLastIndex(), message.toSequenceNr);
236         LOG.debug("{}: adjusted delete to {}", persistenceId, to);
237
238         if (lastDelete < to) {
239             LOG.debug("{}: deleting entries up to {}", persistenceId, to);
240
241             lastDelete = to;
242             final SegmentedJournalWriter<Long> deleteWriter = deleteJournal.writer();
243             final Indexed<Long> entry = deleteWriter.append(lastDelete);
244             deleteWriter.commit(entry.index());
245             dataJournal.writer().commit(lastDelete);
246
247             LOG.debug("{}: compaction started", persistenceId);
248             dataJournal.compact(lastDelete + 1);
249             deleteJournal.compact(entry.index());
250             LOG.debug("{}: compaction finished", persistenceId);
251         } else {
252             LOG.debug("{}: entries up to {} already deleted", persistenceId, lastDelete);
253         }
254
255         message.promise.success(null);
256     }
257
258     @SuppressWarnings("checkstyle:illegalCatch")
259     private void handleReadHighestSequenceNr(final ReadHighestSequenceNr message) {
260         LOG.debug("{}: looking for highest sequence on {}", persistenceId, message);
261         final Long sequence;
262         if (directory.isDirectory()) {
263             ensureOpen();
264             sequence = dataJournal.writer().getLastIndex();
265         } else {
266             sequence = 0L;
267         }
268
269         LOG.debug("{}: highest sequence is {}", message, sequence);
270         message.promise.success(sequence);
271     }
272
273     @SuppressWarnings("checkstyle:illegalCatch")
274     private void handleReplayMessages(final ReplayMessages message) {
275         LOG.debug("{}: replaying messages {}", persistenceId, message);
276         ensureOpen();
277
278         final long from = Long.max(lastDelete + 1, message.fromSequenceNr);
279         LOG.debug("{}: adjusted fromSequenceNr to {}", persistenceId, from);
280
281         try (SegmentedJournalReader<DataJournalEntry> reader = dataJournal.openReader(from)) {
282             int count = 0;
283             while (reader.hasNext() && count < message.max) {
284                 final Indexed<DataJournalEntry> next = reader.next();
285                 if (next.index() > message.toSequenceNr) {
286                     break;
287                 }
288
289                 LOG.trace("{}: replay {}", persistenceId, next);
290                 updateLargestSize(next.size());
291                 final DataJournalEntry entry = next.entry();
292                 verify(entry instanceof FromPersistence, "Unexpected entry %s", entry);
293
294                 final PersistentRepr repr = ((FromPersistence) entry).toRepr(persistenceId, next.index());
295                 LOG.debug("{}: replaying {}", persistenceId, repr);
296                 message.replayCallback.accept(repr);
297                 count++;
298             }
299             LOG.debug("{}: successfully replayed {} entries", persistenceId, count);
300         } catch (Exception e) {
301             LOG.warn("{}: failed to replay messages for {}", persistenceId, message, e);
302             message.promise.failure(e);
303         } finally {
304             message.promise.success(null);
305         }
306     }
307
308     @SuppressWarnings("checkstyle:illegalCatch")
309     private void handleWriteMessages(final WriteMessages message) {
310         ensureOpen();
311
312         final SegmentedJournalWriter<DataJournalEntry> writer = dataJournal.writer();
313         final long startTicks = System.nanoTime();
314         final int count = message.requests.size();
315         final long start = writer.getLastIndex();
316
317         for (int i = 0; i < count; ++i) {
318             final long mark = writer.getLastIndex();
319             try {
320                 writeRequest(writer, message.requests.get(i));
321             } catch (Exception e) {
322                 LOG.warn("{}: failed to write out request", persistenceId, e);
323                 message.results.get(i).success(Optional.of(e));
324                 writer.truncate(mark);
325                 continue;
326             }
327
328             message.results.get(i).success(Optional.empty());
329         }
330         writer.flush();
331         batchWriteTime.update(System.nanoTime() - startTicks, TimeUnit.NANOSECONDS);
332         messageWriteCount.mark(writer.getLastIndex() - start);
333     }
334
335     private void writeRequest(final SegmentedJournalWriter<DataJournalEntry> writer, final AtomicWrite request) {
336         // Cast is needed for Eclipse because of https://bugs.eclipse.org/bugs/show_bug.cgi?id=468276
337         final Iterator<PersistentRepr> it = ((SeqLike<PersistentRepr, ?>) request.payload()).iterator();
338         while (it.hasNext()) {
339             final PersistentRepr repr = it.next();
340             final Object payload = repr.payload();
341             if (!(payload instanceof Serializable)) {
342                 throw new UnsupportedOperationException("Non-serializable payload encountered " + payload.getClass());
343             }
344
345             final int size = writer.append(new ToPersistence(repr)).size();
346             messageSize.update(size);
347             updateLargestSize(size);
348         }
349     }
350
351     private void handleUnknown(final Object message) {
352         LOG.error("{}: Received unknown message {}", persistenceId, message);
353     }
354
355     private void updateLargestSize(final int size) {
356         if (size > largestObservedSize) {
357             largestObservedSize = size;
358         }
359     }
360
361     private void ensureOpen() {
362         if (dataJournal != null) {
363             verifyNotNull(deleteJournal);
364             return;
365         }
366
367         deleteJournal = SegmentedJournal.<Long>builder().withDirectory(directory).withName("delete")
368                 .withNamespace(DELETE_NAMESPACE).withMaxSegmentSize(DELETE_SEGMENT_SIZE).build();
369         final Indexed<Long> lastEntry = deleteJournal.writer().getLastEntry();
370         lastDelete = lastEntry == null ? 0 : lastEntry.entry();
371
372         dataJournal = SegmentedJournal.<DataJournalEntry>builder()
373                 .withStorageLevel(storage).withDirectory(directory).withName("data")
374                 .withNamespace(Namespace.builder()
375                     .register(new DataJournalEntrySerializer(context().system()),
376                         FromPersistence.class, ToPersistence.class)
377                     .build())
378                 .withMaxEntrySize(maxEntrySize).withMaxSegmentSize(maxSegmentSize)
379                 .build();
380         final SegmentedJournalWriter<DataJournalEntry> writer = dataJournal.writer();
381         writer.commit(lastDelete);
382         LOG.debug("{}: journal open with last index {}, deleted to {}", persistenceId, writer.getLastIndex(),
383             lastDelete);
384     }
385 }