Centralize JournalSegmentFile creation
[controller.git] / atomix-storage / src / main / java / io / atomix / storage / journal / SegmentedJournal.java
1 /*
2  * Copyright 2017-2022 Open Networking Foundation and others.  All rights reserved.
3  *
4  * Licensed under the Apache License, Version 2.0 (the "License");
5  * you may not use this file except in compliance with the License.
6  * You may obtain a copy of the License at
7  *
8  * http://www.apache.org/licenses/LICENSE-2.0
9  *
10  * Unless required by applicable law or agreed to in writing, software
11  * distributed under the License is distributed on an "AS IS" BASIS,
12  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13  * See the License for the specific language governing permissions and
14  * limitations under the License.
15  */
16 package io.atomix.storage.journal;
17
18 import static com.google.common.base.Preconditions.checkArgument;
19 import static com.google.common.base.Preconditions.checkState;
20 import static java.util.Objects.requireNonNull;
21
22 import java.io.File;
23 import java.io.IOException;
24 import java.util.Collection;
25 import java.util.Map;
26 import java.util.TreeMap;
27 import java.util.concurrent.ConcurrentHashMap;
28 import java.util.concurrent.ConcurrentNavigableMap;
29 import java.util.concurrent.ConcurrentSkipListMap;
30 import org.slf4j.Logger;
31 import org.slf4j.LoggerFactory;
32
33 /**
34  * Segmented journal.
35  */
36 public final class SegmentedJournal<E> implements Journal<E> {
37   /**
38    * Returns a new Raft log builder.
39    *
40    * @return A new Raft log builder.
41    */
42   public static <E> Builder<E> builder() {
43     return new Builder<>();
44   }
45
46   private static final Logger LOG = LoggerFactory.getLogger(SegmentedJournal.class);
47   private static final int SEGMENT_BUFFER_FACTOR = 3;
48
49   private final String name;
50   private final StorageLevel storageLevel;
51   private final File directory;
52   private final JournalSerializer<E> serializer;
53   private final int maxSegmentSize;
54   private final int maxEntrySize;
55   private final int maxEntriesPerSegment;
56   private final double indexDensity;
57   private final boolean flushOnCommit;
58   private final SegmentedJournalWriter<E> writer;
59   private volatile long commitIndex;
60
61   private final ConcurrentNavigableMap<Long, JournalSegment> segments = new ConcurrentSkipListMap<>();
62   private final Collection<SegmentedJournalReader<?>> readers = ConcurrentHashMap.newKeySet();
63   private JournalSegment currentSegment;
64
65   private volatile boolean open = true;
66
67   public SegmentedJournal(
68       String name,
69       StorageLevel storageLevel,
70       File directory,
71       JournalSerdes namespace,
72       int maxSegmentSize,
73       int maxEntrySize,
74       int maxEntriesPerSegment,
75       double indexDensity,
76       boolean flushOnCommit) {
77     this.name = requireNonNull(name, "name cannot be null");
78     this.storageLevel = requireNonNull(storageLevel, "storageLevel cannot be null");
79     this.directory = requireNonNull(directory, "directory cannot be null");
80     this.serializer = JournalSerializer.wrap(requireNonNull(namespace, "namespace cannot be null"));
81     this.maxSegmentSize = maxSegmentSize;
82     this.maxEntrySize = maxEntrySize;
83     this.maxEntriesPerSegment = maxEntriesPerSegment;
84     this.indexDensity = indexDensity;
85     this.flushOnCommit = flushOnCommit;
86     open();
87     this.writer = new SegmentedJournalWriter<>(this);
88   }
89
90   /**
91    * Returns the segment file name prefix.
92    *
93    * @return The segment file name prefix.
94    */
95   public String name() {
96     return name;
97   }
98
99   /**
100    * Returns the storage directory.
101    * <p>
102    * The storage directory is the directory to which all segments write files. Segment files for multiple logs may be
103    * stored in the storage directory, and files for each log instance will be identified by the {@code prefix} provided
104    * when the log is opened.
105    *
106    * @return The storage directory.
107    */
108   public File directory() {
109     return directory;
110   }
111
112   /**
113    * Returns the storage level.
114    * <p>
115    * The storage level dictates how entries within individual journal segments should be stored.
116    *
117    * @return The storage level.
118    */
119   public StorageLevel storageLevel() {
120     return storageLevel;
121   }
122
123   /**
124    * Returns the maximum journal segment size.
125    * <p>
126    * The maximum segment size dictates the maximum size any segment in a segment may consume in bytes.
127    *
128    * @return The maximum segment size in bytes.
129    */
130   public int maxSegmentSize() {
131     return maxSegmentSize;
132   }
133
134   /**
135    * Returns the maximum journal entry size.
136    * <p>
137    * The maximum entry size dictates the maximum size any entry in the segment may consume in bytes.
138    *
139    * @return the maximum entry size in bytes
140    */
141   public int maxEntrySize() {
142     return maxEntrySize;
143   }
144
145   /**
146    * Returns the maximum number of entries per segment.
147    * <p>
148    * The maximum entries per segment dictates the maximum number of entries that are allowed to be stored in any segment
149    * in a journal.
150    *
151    * @return The maximum number of entries per segment.
152    * @deprecated since 3.0.2
153    */
154   @Deprecated
155   public int maxEntriesPerSegment() {
156     return maxEntriesPerSegment;
157   }
158
159   /**
160    * Returns the collection of journal segments.
161    *
162    * @return the collection of journal segments
163    */
164   public Collection<JournalSegment> segments() {
165     return segments.values();
166   }
167
168   /**
169    * Returns the collection of journal segments with indexes greater than the given index.
170    *
171    * @param index the starting index
172    * @return the journal segments starting with indexes greater than or equal to the given index
173    */
174   public Collection<JournalSegment> segments(long index) {
175     return segments.tailMap(index).values();
176   }
177
178   /**
179    * Returns serializer instance.
180    *
181    * @return serializer instance
182    */
183   JournalSerializer<E> serializer() {
184     return serializer;
185   }
186
187   /**
188    * Returns the total size of the journal.
189    *
190    * @return the total size of the journal
191    */
192   public long size() {
193     return segments.values().stream()
194         .mapToLong(JournalSegment::size)
195         .sum();
196   }
197
198   @Override
199   public JournalWriter<E> writer() {
200     return writer;
201   }
202
203   @Override
204   public JournalReader<E> openReader(long index) {
205     return openReader(index, JournalReader.Mode.ALL);
206   }
207
208   /**
209    * Opens a new Raft log reader with the given reader mode.
210    *
211    * @param index The index from which to begin reading entries.
212    * @param mode The mode in which to read entries.
213    * @return The Raft log reader.
214    */
215   @Override
216   public JournalReader<E> openReader(long index, JournalReader.Mode mode) {
217     final var segment = getSegment(index);
218     final var reader = switch (mode) {
219       case ALL -> new SegmentedJournalReader<>(this, segment);
220       case COMMITS -> new CommitsSegmentJournalReader<>(this, segment);
221     };
222
223     // Forward reader to specified index
224     long next = reader.getNextIndex();
225     while (index > next && reader.tryAdvance()) {
226         next = reader.getNextIndex();
227     }
228
229     readers.add(reader);
230     return reader;
231   }
232
233   /**
234    * Opens the segments.
235    */
236   private synchronized void open() {
237     // Load existing log segments from disk.
238     for (var segment : loadSegments()) {
239       segments.put(segment.firstIndex(), segment);
240     }
241
242     // If a segment doesn't already exist, create an initial segment starting at index 1.
243     if (!segments.isEmpty()) {
244       currentSegment = segments.lastEntry().getValue();
245     } else {
246       currentSegment = createSegment(1, 1);
247       segments.put(1L, currentSegment);
248     }
249   }
250
251   /**
252    * Asserts that the manager is open.
253    *
254    * @throws IllegalStateException if the segment manager is not open
255    */
256   private void assertOpen() {
257     checkState(currentSegment != null, "journal not open");
258   }
259
260   /**
261    * Asserts that enough disk space is available to allocate a new segment.
262    */
263   private void assertDiskSpace() {
264     if (directory().getUsableSpace() < maxSegmentSize() * SEGMENT_BUFFER_FACTOR) {
265       throw new StorageException.OutOfDiskSpace("Not enough space to allocate a new journal segment");
266     }
267   }
268
269   /**
270    * Resets the current segment, creating a new segment if necessary.
271    */
272   private synchronized void resetCurrentSegment() {
273     final var lastSegment = getLastSegment();
274     if (lastSegment == null) {
275       currentSegment = createSegment(1, 1);
276       segments.put(1L, currentSegment);
277     } else {
278       currentSegment = lastSegment;
279     }
280   }
281
282   /**
283    * Resets and returns the first segment in the journal.
284    *
285    * @param index the starting index of the journal
286    * @return the first segment
287    */
288   JournalSegment resetSegments(long index) {
289     assertOpen();
290
291     // If the index already equals the first segment index, skip the reset.
292     final var firstSegment = getFirstSegment();
293     if (index == firstSegment.firstIndex()) {
294       return firstSegment;
295     }
296
297     segments.values().forEach(JournalSegment::delete);
298     segments.clear();
299
300     currentSegment = createSegment(1, index);
301     segments.put(index, currentSegment);
302     return currentSegment;
303   }
304
305   /**
306    * Returns the first segment in the log.
307    *
308    * @throws IllegalStateException if the segment manager is not open
309    */
310   JournalSegment getFirstSegment() {
311     assertOpen();
312     Map.Entry<Long, JournalSegment> segment = segments.firstEntry();
313     return segment != null ? segment.getValue() : null;
314   }
315
316   /**
317    * Returns the last segment in the log.
318    *
319    * @throws IllegalStateException if the segment manager is not open
320    */
321   JournalSegment getLastSegment() {
322     assertOpen();
323     Map.Entry<Long, JournalSegment> segment = segments.lastEntry();
324     return segment != null ? segment.getValue() : null;
325   }
326
327   /**
328    * Creates and returns the next segment.
329    *
330    * @return The next segment.
331    * @throws IllegalStateException if the segment manager is not open
332    */
333   synchronized JournalSegment getNextSegment() {
334     assertOpen();
335     assertDiskSpace();
336
337     final var index = currentSegment.lastIndex() + 1;
338     final var lastSegment = getLastSegment();
339     currentSegment = createSegment(lastSegment != null ? lastSegment.file().descriptor().id() + 1 : 1, index);
340     segments.put(index, currentSegment);
341     return currentSegment;
342   }
343
344   /**
345    * Returns the segment following the segment with the given ID.
346    *
347    * @param index The segment index with which to look up the next segment.
348    * @return The next segment for the given index.
349    */
350   JournalSegment getNextSegment(long index) {
351     Map.Entry<Long, JournalSegment> nextSegment = segments.higherEntry(index);
352     return nextSegment != null ? nextSegment.getValue() : null;
353   }
354
355   /**
356    * Returns the segment for the given index.
357    *
358    * @param index The index for which to return the segment.
359    * @throws IllegalStateException if the segment manager is not open
360    */
361   synchronized JournalSegment getSegment(long index) {
362     assertOpen();
363     // Check if the current segment contains the given index first in order to prevent an unnecessary map lookup.
364     if (currentSegment != null && index > currentSegment.firstIndex()) {
365       return currentSegment;
366     }
367
368     // If the index is in another segment, get the entry with the next lowest first index.
369     Map.Entry<Long, JournalSegment> segment = segments.floorEntry(index);
370     if (segment != null) {
371       return segment.getValue();
372     }
373     return getFirstSegment();
374   }
375
376   /**
377    * Removes a segment.
378    *
379    * @param segment The segment to remove.
380    */
381   synchronized void removeSegment(JournalSegment segment) {
382     segments.remove(segment.firstIndex());
383     segment.delete();
384     resetCurrentSegment();
385   }
386
387   /**
388    * Creates a new segment.
389    */
390   JournalSegment createSegment(long id, long index) {
391     final JournalSegmentFile file;
392     try {
393       file = JournalSegmentFile.createNew(name, directory, JournalSegmentDescriptor.builder()
394           .withId(id)
395           .withIndex(index)
396           .withMaxSegmentSize(maxSegmentSize)
397           .withMaxEntries(maxEntriesPerSegment)
398           .withUpdated(System.currentTimeMillis())
399           .build());
400     } catch (IOException e) {
401       throw new StorageException(e);
402     }
403
404     final var segment = new JournalSegment(file, storageLevel, maxEntrySize, indexDensity);
405     LOG.debug("Created segment: {}", segment);
406     return segment;
407   }
408
409   /**
410    * Loads all segments from disk.
411    *
412    * @return A collection of segments for the log.
413    */
414   protected Collection<JournalSegment> loadSegments() {
415     // Ensure log directories are created.
416     directory.mkdirs();
417
418     final var segments = new TreeMap<Long, JournalSegment>();
419
420     // Iterate through all files in the log directory.
421     for (var file : directory.listFiles(File::isFile)) {
422
423       // If the file looks like a segment file, attempt to load the segment.
424       if (JournalSegmentFile.isSegmentFile(name, file)) {
425         final JournalSegmentFile segmentFile;
426         try {
427           segmentFile = JournalSegmentFile.openExisting(file.toPath());
428         } catch (IOException e) {
429           throw new StorageException(e);
430         }
431
432         // Load the segment.
433         LOG.debug("Loaded disk segment: {} ({})", segmentFile.descriptor().id(), segmentFile.path());
434
435         // Add the segment to the segments list.
436         final var segment = new JournalSegment(segmentFile, storageLevel, maxEntrySize, indexDensity);
437         segments.put(segment.firstIndex(), segment);
438       }
439     }
440
441     // Verify that all the segments in the log align with one another.
442     JournalSegment previousSegment = null;
443     boolean corrupted = false;
444     final var iterator = segments.entrySet().iterator();
445     while (iterator.hasNext()) {
446       final var segment = iterator.next().getValue();
447       if (previousSegment != null && previousSegment.lastIndex() != segment.firstIndex() - 1) {
448         LOG.warn("Journal is inconsistent. {} is not aligned with prior segment {}", segment.file().path(),
449             previousSegment.file().path());
450         corrupted = true;
451       }
452       if (corrupted) {
453         segment.delete();
454         iterator.remove();
455       }
456       previousSegment = segment;
457     }
458
459     return segments.values();
460   }
461
462   /**
463    * Resets journal readers to the given head.
464    *
465    * @param index The index at which to reset readers.
466    */
467   void resetHead(long index) {
468     for (var reader : readers) {
469       if (reader.getNextIndex() < index) {
470         reader.reset(index);
471       }
472     }
473   }
474
475   /**
476    * Resets journal readers to the given tail.
477    *
478    * @param index The index at which to reset readers.
479    */
480   void resetTail(long index) {
481     for (var reader : readers) {
482       if (reader.getNextIndex() >= index) {
483         reader.reset(index);
484       }
485     }
486   }
487
488   void closeReader(SegmentedJournalReader<E> reader) {
489     readers.remove(reader);
490   }
491
492   @Override
493   public boolean isOpen() {
494     return open;
495   }
496
497   /**
498    * Returns a boolean indicating whether a segment can be removed from the journal prior to the given index.
499    *
500    * @param index the index from which to remove segments
501    * @return indicates whether a segment can be removed from the journal
502    */
503   public boolean isCompactable(long index) {
504     Map.Entry<Long, JournalSegment> segmentEntry = segments.floorEntry(index);
505     return segmentEntry != null && segments.headMap(segmentEntry.getValue().firstIndex()).size() > 0;
506   }
507
508   /**
509    * Returns the index of the last segment in the log.
510    *
511    * @param index the compaction index
512    * @return the starting index of the last segment in the log
513    */
514   public long getCompactableIndex(long index) {
515     Map.Entry<Long, JournalSegment> segmentEntry = segments.floorEntry(index);
516     return segmentEntry != null ? segmentEntry.getValue().firstIndex() : 0;
517   }
518
519   /**
520    * Compacts the journal up to the given index.
521    * <p>
522    * The semantics of compaction are not specified by this interface.
523    *
524    * @param index The index up to which to compact the journal.
525    */
526   public void compact(long index) {
527     final var segmentEntry = segments.floorEntry(index);
528     if (segmentEntry != null) {
529       final var compactSegments = segments.headMap(segmentEntry.getValue().firstIndex());
530       if (!compactSegments.isEmpty()) {
531         LOG.debug("{} - Compacting {} segment(s)", name, compactSegments.size());
532         compactSegments.values().forEach(JournalSegment::delete);
533         compactSegments.clear();
534         resetHead(segmentEntry.getValue().firstIndex());
535       }
536     }
537   }
538
539   @Override
540   public void close() {
541     segments.values().forEach(JournalSegment::close);
542     currentSegment = null;
543     open = false;
544   }
545
546   /**
547    * Returns whether {@code flushOnCommit} is enabled for the log.
548    *
549    * @return Indicates whether {@code flushOnCommit} is enabled for the log.
550    */
551   boolean isFlushOnCommit() {
552     return flushOnCommit;
553   }
554
555   /**
556    * Commits entries up to the given index.
557    *
558    * @param index The index up to which to commit entries.
559    */
560   void setCommitIndex(long index) {
561     this.commitIndex = index;
562   }
563
564   /**
565    * Returns the Raft log commit index.
566    *
567    * @return The Raft log commit index.
568    */
569   long getCommitIndex() {
570     return commitIndex;
571   }
572
573   /**
574    * Raft log builder.
575    */
576   public static final class Builder<E> {
577     private static final boolean DEFAULT_FLUSH_ON_COMMIT = false;
578     private static final String DEFAULT_NAME = "atomix";
579     private static final String DEFAULT_DIRECTORY = System.getProperty("user.dir");
580     private static final int DEFAULT_MAX_SEGMENT_SIZE = 1024 * 1024 * 32;
581     private static final int DEFAULT_MAX_ENTRY_SIZE = 1024 * 1024;
582     private static final int DEFAULT_MAX_ENTRIES_PER_SEGMENT = 1024 * 1024;
583     private static final double DEFAULT_INDEX_DENSITY = .005;
584
585     private String name = DEFAULT_NAME;
586     private StorageLevel storageLevel = StorageLevel.DISK;
587     private File directory = new File(DEFAULT_DIRECTORY);
588     private JournalSerdes namespace;
589     private int maxSegmentSize = DEFAULT_MAX_SEGMENT_SIZE;
590     private int maxEntrySize = DEFAULT_MAX_ENTRY_SIZE;
591     private int maxEntriesPerSegment = DEFAULT_MAX_ENTRIES_PER_SEGMENT;
592     private double indexDensity = DEFAULT_INDEX_DENSITY;
593     private boolean flushOnCommit = DEFAULT_FLUSH_ON_COMMIT;
594
595     Builder() {
596       // Hidden on purpose
597     }
598
599     /**
600      * Sets the storage name.
601      *
602      * @param name The storage name.
603      * @return The storage builder.
604      */
605     public Builder<E> withName(String name) {
606       this.name = requireNonNull(name, "name cannot be null");
607       return this;
608     }
609
610     /**
611      * Sets the log storage level, returning the builder for method chaining.
612      * <p>
613      * The storage level indicates how individual entries should be persisted in the journal.
614      *
615      * @param storageLevel The log storage level.
616      * @return The storage builder.
617      */
618     public Builder<E> withStorageLevel(StorageLevel storageLevel) {
619       this.storageLevel = requireNonNull(storageLevel, "storageLevel cannot be null");
620       return this;
621     }
622
623     /**
624      * Sets the log directory, returning the builder for method chaining.
625      * <p>
626      * The log will write segment files into the provided directory.
627      *
628      * @param directory The log directory.
629      * @return The storage builder.
630      * @throws NullPointerException If the {@code directory} is {@code null}
631      */
632     public Builder<E> withDirectory(String directory) {
633       return withDirectory(new File(requireNonNull(directory, "directory cannot be null")));
634     }
635
636     /**
637      * Sets the log directory, returning the builder for method chaining.
638      * <p>
639      * The log will write segment files into the provided directory.
640      *
641      * @param directory The log directory.
642      * @return The storage builder.
643      * @throws NullPointerException If the {@code directory} is {@code null}
644      */
645     public Builder<E> withDirectory(File directory) {
646       this.directory = requireNonNull(directory, "directory cannot be null");
647       return this;
648     }
649
650     /**
651      * Sets the journal namespace, returning the builder for method chaining.
652      *
653      * @param namespace The journal serializer.
654      * @return The journal builder.
655      */
656     public Builder<E> withNamespace(JournalSerdes namespace) {
657       this.namespace = requireNonNull(namespace, "namespace cannot be null");
658       return this;
659     }
660
661     /**
662      * Sets the maximum segment size in bytes, returning the builder for method chaining.
663      * <p>
664      * The maximum segment size dictates when logs should roll over to new segments. As entries are written to a segment
665      * of the log, once the size of the segment surpasses the configured maximum segment size, the log will create a new
666      * segment and append new entries to that segment.
667      * <p>
668      * By default, the maximum segment size is {@code 1024 * 1024 * 32}.
669      *
670      * @param maxSegmentSize The maximum segment size in bytes.
671      * @return The storage builder.
672      * @throws IllegalArgumentException If the {@code maxSegmentSize} is not positive
673      */
674     public Builder<E> withMaxSegmentSize(int maxSegmentSize) {
675       checkArgument(maxSegmentSize > JournalSegmentDescriptor.BYTES,
676           "maxSegmentSize must be greater than " + JournalSegmentDescriptor.BYTES);
677       this.maxSegmentSize = maxSegmentSize;
678       return this;
679     }
680
681     /**
682      * Sets the maximum entry size in bytes, returning the builder for method chaining.
683      *
684      * @param maxEntrySize the maximum entry size in bytes
685      * @return the storage builder
686      * @throws IllegalArgumentException if the {@code maxEntrySize} is not positive
687      */
688     public Builder<E> withMaxEntrySize(int maxEntrySize) {
689       checkArgument(maxEntrySize > 0, "maxEntrySize must be positive");
690       this.maxEntrySize = maxEntrySize;
691       return this;
692     }
693
694     /**
695      * Sets the maximum number of allows entries per segment, returning the builder for method chaining.
696      * <p>
697      * The maximum entry count dictates when logs should roll over to new segments. As entries are written to a segment
698      * of the log, if the entry count in that segment meets the configured maximum entry count, the log will create a
699      * new segment and append new entries to that segment.
700      * <p>
701      * By default, the maximum entries per segment is {@code 1024 * 1024}.
702      *
703      * @param maxEntriesPerSegment The maximum number of entries allowed per segment.
704      * @return The storage builder.
705      * @throws IllegalArgumentException If the {@code maxEntriesPerSegment} not greater than the default max entries
706      *     per segment
707      * @deprecated since 3.0.2
708      */
709     @Deprecated
710     public Builder<E> withMaxEntriesPerSegment(int maxEntriesPerSegment) {
711       checkArgument(maxEntriesPerSegment > 0, "max entries per segment must be positive");
712       checkArgument(maxEntriesPerSegment <= DEFAULT_MAX_ENTRIES_PER_SEGMENT,
713           "max entries per segment cannot be greater than " + DEFAULT_MAX_ENTRIES_PER_SEGMENT);
714       this.maxEntriesPerSegment = maxEntriesPerSegment;
715       return this;
716     }
717
718     /**
719      * Sets the journal index density.
720      * <p>
721      * The index density is the frequency at which the position of entries written to the journal will be recorded in an
722      * in-memory index for faster seeking.
723      *
724      * @param indexDensity the index density
725      * @return the journal builder
726      * @throws IllegalArgumentException if the density is not between 0 and 1
727      */
728     public Builder<E> withIndexDensity(double indexDensity) {
729       checkArgument(indexDensity > 0 && indexDensity < 1, "index density must be between 0 and 1");
730       this.indexDensity = indexDensity;
731       return this;
732     }
733
734     /**
735      * Enables flushing buffers to disk when entries are committed to a segment, returning the builder for method
736      * chaining.
737      * <p>
738      * When flush-on-commit is enabled, log entry buffers will be automatically flushed to disk each time an entry is
739      * committed in a given segment.
740      *
741      * @return The storage builder.
742      */
743     public Builder<E> withFlushOnCommit() {
744       return withFlushOnCommit(true);
745     }
746
747     /**
748      * Sets whether to flush buffers to disk when entries are committed to a segment, returning the builder for method
749      * chaining.
750      * <p>
751      * When flush-on-commit is enabled, log entry buffers will be automatically flushed to disk each time an entry is
752      * committed in a given segment.
753      *
754      * @param flushOnCommit Whether to flush buffers to disk when entries are committed to a segment.
755      * @return The storage builder.
756      */
757     public Builder<E> withFlushOnCommit(boolean flushOnCommit) {
758       this.flushOnCommit = flushOnCommit;
759       return this;
760     }
761
762     /**
763      * Build the {@link SegmentedJournal}.
764      *
765      * @return A new {@link SegmentedJournal}.
766      */
767     public SegmentedJournal<E> build() {
768       return new SegmentedJournal<>(
769           name,
770           storageLevel,
771           directory,
772           namespace,
773           maxSegmentSize,
774           maxEntrySize,
775           maxEntriesPerSegment,
776           indexDensity,
777           flushOnCommit);
778     }
779   }
780 }