2 * Copyright 2017-present Open Networking Foundation
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
8 * http://www.apache.org/licenses/LICENSE-2.0
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.
16 package io.atomix.storage.journal;
19 import java.io.IOException;
20 import java.io.RandomAccessFile;
21 import java.nio.ByteBuffer;
22 import java.nio.channels.FileChannel;
23 import java.nio.file.StandardOpenOption;
24 import java.util.Collection;
25 import java.util.Iterator;
27 import java.util.SortedMap;
28 import java.util.TreeMap;
29 import java.util.concurrent.ConcurrentHashMap;
30 import java.util.concurrent.ConcurrentNavigableMap;
31 import java.util.concurrent.ConcurrentSkipListMap;
32 import org.slf4j.Logger;
33 import org.slf4j.LoggerFactory;
35 import static com.google.common.base.Preconditions.checkArgument;
36 import static com.google.common.base.Preconditions.checkState;
37 import static java.util.Objects.requireNonNull;
42 public final class SegmentedJournal<E> implements Journal<E> {
44 * Returns a new Raft log builder.
46 * @return A new Raft log builder.
48 public static <E> Builder<E> builder() {
49 return new Builder<>();
52 private static final int SEGMENT_BUFFER_FACTOR = 3;
54 private final Logger log = LoggerFactory.getLogger(getClass());
55 private final String name;
56 private final StorageLevel storageLevel;
57 private final File directory;
58 private final JournalSerdes namespace;
59 private final int maxSegmentSize;
60 private final int maxEntrySize;
61 private final int maxEntriesPerSegment;
62 private final double indexDensity;
63 private final boolean flushOnCommit;
64 private final SegmentedJournalWriter<E> writer;
65 private volatile long commitIndex;
67 private final ConcurrentNavigableMap<Long, JournalSegment<E>> segments = new ConcurrentSkipListMap<>();
68 private final Collection<SegmentedJournalReader<E>> readers = ConcurrentHashMap.newKeySet();
69 private JournalSegment<E> currentSegment;
71 private volatile boolean open = true;
73 public SegmentedJournal(
75 StorageLevel storageLevel,
77 JournalSerdes namespace,
80 int maxEntriesPerSegment,
82 boolean flushOnCommit) {
83 this.name = requireNonNull(name, "name cannot be null");
84 this.storageLevel = requireNonNull(storageLevel, "storageLevel cannot be null");
85 this.directory = requireNonNull(directory, "directory cannot be null");
86 this.namespace = requireNonNull(namespace, "namespace cannot be null");
87 this.maxSegmentSize = maxSegmentSize;
88 this.maxEntrySize = maxEntrySize;
89 this.maxEntriesPerSegment = maxEntriesPerSegment;
90 this.indexDensity = indexDensity;
91 this.flushOnCommit = flushOnCommit;
93 this.writer = new SegmentedJournalWriter<>(this);
97 * Returns the segment file name prefix.
99 * @return The segment file name prefix.
101 public String name() {
106 * Returns the storage directory.
108 * The storage directory is the directory to which all segments write files. Segment files for multiple logs may be
109 * stored in the storage directory, and files for each log instance will be identified by the {@code prefix} provided
110 * when the log is opened.
112 * @return The storage directory.
114 public File directory() {
119 * Returns the storage level.
121 * The storage level dictates how entries within individual journal segments should be stored.
123 * @return The storage level.
125 public StorageLevel storageLevel() {
130 * Returns the maximum journal segment size.
132 * The maximum segment size dictates the maximum size any segment in a segment may consume in bytes.
134 * @return The maximum segment size in bytes.
136 public int maxSegmentSize() {
137 return maxSegmentSize;
141 * Returns the maximum journal entry size.
143 * The maximum entry size dictates the maximum size any entry in the segment may consume in bytes.
145 * @return the maximum entry size in bytes
147 public int maxEntrySize() {
152 * Returns the maximum number of entries per segment.
154 * The maximum entries per segment dictates the maximum number of entries that are allowed to be stored in any segment
157 * @return The maximum number of entries per segment.
158 * @deprecated since 3.0.2
161 public int maxEntriesPerSegment() {
162 return maxEntriesPerSegment;
166 * Returns the collection of journal segments.
168 * @return the collection of journal segments
170 public Collection<JournalSegment<E>> segments() {
171 return segments.values();
175 * Returns the collection of journal segments with indexes greater than the given index.
177 * @param index the starting index
178 * @return the journal segments starting with indexes greater than or equal to the given index
180 public Collection<JournalSegment<E>> segments(long index) {
181 return segments.tailMap(index).values();
185 * Returns the total size of the journal.
187 * @return the total size of the journal
190 return segments.values().stream()
191 .mapToLong(segment -> segment.size())
196 public JournalWriter<E> writer() {
201 public JournalReader<E> openReader(long index) {
202 return openReader(index, JournalReader.Mode.ALL);
206 * Opens a new Raft log reader with the given reader mode.
208 * @param index The index from which to begin reading entries.
209 * @param mode The mode in which to read entries.
210 * @return The Raft log reader.
212 public JournalReader<E> openReader(long index, JournalReader.Mode mode) {
213 final var reader = new SegmentedJournalReader<>(this, index, mode);
219 * Opens the segments.
221 private synchronized void open() {
222 // Load existing log segments from disk.
223 for (JournalSegment<E> segment : loadSegments()) {
224 segments.put(segment.descriptor().index(), segment);
227 // If a segment doesn't already exist, create an initial segment starting at index 1.
228 if (!segments.isEmpty()) {
229 currentSegment = segments.lastEntry().getValue();
231 JournalSegmentDescriptor descriptor = JournalSegmentDescriptor.builder()
234 .withMaxSegmentSize(maxSegmentSize)
235 .withMaxEntries(maxEntriesPerSegment)
238 currentSegment = createSegment(descriptor);
239 currentSegment.descriptor().update(System.currentTimeMillis());
241 segments.put(1L, currentSegment);
246 * Asserts that the manager is open.
248 * @throws IllegalStateException if the segment manager is not open
250 private void assertOpen() {
251 checkState(currentSegment != null, "journal not open");
255 * Asserts that enough disk space is available to allocate a new segment.
257 private void assertDiskSpace() {
258 if (directory().getUsableSpace() < maxSegmentSize() * SEGMENT_BUFFER_FACTOR) {
259 throw new StorageException.OutOfDiskSpace("Not enough space to allocate a new journal segment");
264 * Resets the current segment, creating a new segment if necessary.
266 private synchronized void resetCurrentSegment() {
267 JournalSegment<E> lastSegment = getLastSegment();
268 if (lastSegment != null) {
269 currentSegment = lastSegment;
271 JournalSegmentDescriptor descriptor = JournalSegmentDescriptor.builder()
274 .withMaxSegmentSize(maxSegmentSize)
275 .withMaxEntries(maxEntriesPerSegment)
278 currentSegment = createSegment(descriptor);
280 segments.put(1L, currentSegment);
285 * Resets and returns the first segment in the journal.
287 * @param index the starting index of the journal
288 * @return the first segment
290 JournalSegment<E> resetSegments(long index) {
293 // If the index already equals the first segment index, skip the reset.
294 JournalSegment<E> firstSegment = getFirstSegment();
295 if (index == firstSegment.index()) {
299 for (JournalSegment<E> segment : segments.values()) {
305 JournalSegmentDescriptor descriptor = JournalSegmentDescriptor.builder()
308 .withMaxSegmentSize(maxSegmentSize)
309 .withMaxEntries(maxEntriesPerSegment)
311 currentSegment = createSegment(descriptor);
312 segments.put(index, currentSegment);
313 return currentSegment;
317 * Returns the first segment in the log.
319 * @throws IllegalStateException if the segment manager is not open
321 JournalSegment<E> getFirstSegment() {
323 Map.Entry<Long, JournalSegment<E>> segment = segments.firstEntry();
324 return segment != null ? segment.getValue() : null;
328 * Returns the last segment in the log.
330 * @throws IllegalStateException if the segment manager is not open
332 JournalSegment<E> getLastSegment() {
334 Map.Entry<Long, JournalSegment<E>> segment = segments.lastEntry();
335 return segment != null ? segment.getValue() : null;
339 * Creates and returns the next segment.
341 * @return The next segment.
342 * @throws IllegalStateException if the segment manager is not open
344 synchronized JournalSegment<E> getNextSegment() {
348 JournalSegment<E> lastSegment = getLastSegment();
349 JournalSegmentDescriptor descriptor = JournalSegmentDescriptor.builder()
350 .withId(lastSegment != null ? lastSegment.descriptor().id() + 1 : 1)
351 .withIndex(currentSegment.lastIndex() + 1)
352 .withMaxSegmentSize(maxSegmentSize)
353 .withMaxEntries(maxEntriesPerSegment)
356 currentSegment = createSegment(descriptor);
358 segments.put(descriptor.index(), currentSegment);
359 return currentSegment;
363 * Returns the segment following the segment with the given ID.
365 * @param index The segment index with which to look up the next segment.
366 * @return The next segment for the given index.
368 JournalSegment<E> getNextSegment(long index) {
369 Map.Entry<Long, JournalSegment<E>> nextSegment = segments.higherEntry(index);
370 return nextSegment != null ? nextSegment.getValue() : null;
374 * Returns the segment for the given index.
376 * @param index The index for which to return the segment.
377 * @throws IllegalStateException if the segment manager is not open
379 synchronized JournalSegment<E> getSegment(long index) {
381 // Check if the current segment contains the given index first in order to prevent an unnecessary map lookup.
382 if (currentSegment != null && index > currentSegment.index()) {
383 return currentSegment;
386 // If the index is in another segment, get the entry with the next lowest first index.
387 Map.Entry<Long, JournalSegment<E>> segment = segments.floorEntry(index);
388 if (segment != null) {
389 return segment.getValue();
391 return getFirstSegment();
397 * @param segment The segment to remove.
399 synchronized void removeSegment(JournalSegment<E> segment) {
400 segments.remove(segment.index());
403 resetCurrentSegment();
407 * Creates a new segment.
409 JournalSegment<E> createSegment(JournalSegmentDescriptor descriptor) {
410 File segmentFile = JournalSegmentFile.createSegmentFile(name, directory, descriptor.id());
412 RandomAccessFile raf;
415 raf = new RandomAccessFile(segmentFile, "rw");
416 raf.setLength(descriptor.maxSegmentSize());
417 channel = raf.getChannel();
418 } catch (IOException e) {
419 throw new StorageException(e);
422 ByteBuffer buffer = ByteBuffer.allocate(JournalSegmentDescriptor.BYTES);
423 descriptor.copyTo(buffer);
426 channel.write(buffer);
427 } catch (IOException e) {
428 throw new StorageException(e);
433 } catch (IOException e) {
436 JournalSegment<E> segment = newSegment(new JournalSegmentFile(segmentFile), descriptor);
437 log.debug("Created segment: {}", segment);
442 * Creates a new segment instance.
444 * @param segmentFile The segment file.
445 * @param descriptor The segment descriptor.
446 * @return The segment instance.
448 protected JournalSegment<E> newSegment(JournalSegmentFile segmentFile, JournalSegmentDescriptor descriptor) {
449 return new JournalSegment<>(segmentFile, descriptor, storageLevel, maxEntrySize, indexDensity, namespace);
455 private JournalSegment<E> loadSegment(long segmentId) {
456 File segmentFile = JournalSegmentFile.createSegmentFile(name, directory, segmentId);
457 ByteBuffer buffer = ByteBuffer.allocate(JournalSegmentDescriptor.BYTES);
458 try (FileChannel channel = openChannel(segmentFile)) {
459 channel.read(buffer);
461 JournalSegmentDescriptor descriptor = new JournalSegmentDescriptor(buffer);
462 JournalSegment<E> segment = newSegment(new JournalSegmentFile(segmentFile), descriptor);
463 log.debug("Loaded disk segment: {} ({})", descriptor.id(), segmentFile.getName());
465 } catch (IOException e) {
466 throw new StorageException(e);
470 private FileChannel openChannel(File file) {
472 return FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.READ, StandardOpenOption.WRITE);
473 } catch (IOException e) {
474 throw new StorageException(e);
479 * Loads all segments from disk.
481 * @return A collection of segments for the log.
483 protected Collection<JournalSegment<E>> loadSegments() {
484 // Ensure log directories are created.
487 TreeMap<Long, JournalSegment<E>> segments = new TreeMap<>();
489 // Iterate through all files in the log directory.
490 for (File file : directory.listFiles(File::isFile)) {
492 // If the file looks like a segment file, attempt to load the segment.
493 if (JournalSegmentFile.isSegmentFile(name, file)) {
494 JournalSegmentFile segmentFile = new JournalSegmentFile(file);
495 ByteBuffer buffer = ByteBuffer.allocate(JournalSegmentDescriptor.BYTES);
496 try (FileChannel channel = openChannel(file)) {
497 channel.read(buffer);
499 } catch (IOException e) {
500 throw new StorageException(e);
503 JournalSegmentDescriptor descriptor = new JournalSegmentDescriptor(buffer);
506 JournalSegment<E> segment = loadSegment(descriptor.id());
508 // Add the segment to the segments list.
509 log.debug("Found segment: {} ({})", segment.descriptor().id(), segmentFile.file().getName());
510 segments.put(segment.index(), segment);
514 // Verify that all the segments in the log align with one another.
515 JournalSegment<E> previousSegment = null;
516 boolean corrupted = false;
517 Iterator<Map.Entry<Long, JournalSegment<E>>> iterator = segments.entrySet().iterator();
518 while (iterator.hasNext()) {
519 JournalSegment<E> segment = iterator.next().getValue();
520 if (previousSegment != null && previousSegment.lastIndex() != segment.index() - 1) {
521 log.warn("Journal is inconsistent. {} is not aligned with prior segment {}", segment.file().file(), previousSegment.file().file());
529 previousSegment = segment;
532 return segments.values();
536 * Resets journal readers to the given head.
538 * @param index The index at which to reset readers.
540 void resetHead(long index) {
541 for (SegmentedJournalReader<E> reader : readers) {
542 if (reader.getNextIndex() < index) {
549 * Resets journal readers to the given tail.
551 * @param index The index at which to reset readers.
553 void resetTail(long index) {
554 for (SegmentedJournalReader<E> reader : readers) {
555 if (reader.getNextIndex() >= index) {
561 void closeReader(SegmentedJournalReader<E> reader) {
562 readers.remove(reader);
566 public boolean isOpen() {
571 * Returns a boolean indicating whether a segment can be removed from the journal prior to the given index.
573 * @param index the index from which to remove segments
574 * @return indicates whether a segment can be removed from the journal
576 public boolean isCompactable(long index) {
577 Map.Entry<Long, JournalSegment<E>> segmentEntry = segments.floorEntry(index);
578 return segmentEntry != null && segments.headMap(segmentEntry.getValue().index()).size() > 0;
582 * Returns the index of the last segment in the log.
584 * @param index the compaction index
585 * @return the starting index of the last segment in the log
587 public long getCompactableIndex(long index) {
588 Map.Entry<Long, JournalSegment<E>> segmentEntry = segments.floorEntry(index);
589 return segmentEntry != null ? segmentEntry.getValue().index() : 0;
593 * Compacts the journal up to the given index.
595 * The semantics of compaction are not specified by this interface.
597 * @param index The index up to which to compact the journal.
599 public void compact(long index) {
600 Map.Entry<Long, JournalSegment<E>> segmentEntry = segments.floorEntry(index);
601 if (segmentEntry != null) {
602 SortedMap<Long, JournalSegment<E>> compactSegments = segments.headMap(segmentEntry.getValue().index());
603 if (!compactSegments.isEmpty()) {
604 log.debug("{} - Compacting {} segment(s)", name, compactSegments.size());
605 for (JournalSegment<E> segment : compactSegments.values()) {
606 log.trace("Deleting segment: {}", segment);
610 compactSegments.clear();
611 resetHead(segmentEntry.getValue().index());
617 public void close() {
618 segments.values().forEach(segment -> {
619 log.debug("Closing segment: {}", segment);
622 currentSegment = null;
627 * Returns whether {@code flushOnCommit} is enabled for the log.
629 * @return Indicates whether {@code flushOnCommit} is enabled for the log.
631 boolean isFlushOnCommit() {
632 return flushOnCommit;
636 * Commits entries up to the given index.
638 * @param index The index up to which to commit entries.
640 void setCommitIndex(long index) {
641 this.commitIndex = index;
645 * Returns the Raft log commit index.
647 * @return The Raft log commit index.
649 long getCommitIndex() {
656 public static final class Builder<E> {
657 private static final boolean DEFAULT_FLUSH_ON_COMMIT = false;
658 private static final String DEFAULT_NAME = "atomix";
659 private static final String DEFAULT_DIRECTORY = System.getProperty("user.dir");
660 private static final int DEFAULT_MAX_SEGMENT_SIZE = 1024 * 1024 * 32;
661 private static final int DEFAULT_MAX_ENTRY_SIZE = 1024 * 1024;
662 private static final int DEFAULT_MAX_ENTRIES_PER_SEGMENT = 1024 * 1024;
663 private static final double DEFAULT_INDEX_DENSITY = .005;
665 private String name = DEFAULT_NAME;
666 private StorageLevel storageLevel = StorageLevel.DISK;
667 private File directory = new File(DEFAULT_DIRECTORY);
668 private JournalSerdes namespace;
669 private int maxSegmentSize = DEFAULT_MAX_SEGMENT_SIZE;
670 private int maxEntrySize = DEFAULT_MAX_ENTRY_SIZE;
671 private int maxEntriesPerSegment = DEFAULT_MAX_ENTRIES_PER_SEGMENT;
672 private double indexDensity = DEFAULT_INDEX_DENSITY;
673 private boolean flushOnCommit = DEFAULT_FLUSH_ON_COMMIT;
675 protected Builder() {
679 * Sets the storage name.
681 * @param name The storage name.
682 * @return The storage builder.
684 public Builder<E> withName(String name) {
685 this.name = requireNonNull(name, "name cannot be null");
690 * Sets the log storage level, returning the builder for method chaining.
692 * The storage level indicates how individual entries should be persisted in the journal.
694 * @param storageLevel The log storage level.
695 * @return The storage builder.
697 public Builder<E> withStorageLevel(StorageLevel storageLevel) {
698 this.storageLevel = requireNonNull(storageLevel, "storageLevel cannot be null");
703 * Sets the log directory, returning the builder for method chaining.
705 * The log will write segment files into the provided directory.
707 * @param directory The log directory.
708 * @return The storage builder.
709 * @throws NullPointerException If the {@code directory} is {@code null}
711 public Builder<E> withDirectory(String directory) {
712 return withDirectory(new File(requireNonNull(directory, "directory cannot be null")));
716 * Sets the log directory, returning the builder for method chaining.
718 * The log will write segment files into the provided directory.
720 * @param directory The log directory.
721 * @return The storage builder.
722 * @throws NullPointerException If the {@code directory} is {@code null}
724 public Builder<E> withDirectory(File directory) {
725 this.directory = requireNonNull(directory, "directory cannot be null");
730 * Sets the journal namespace, returning the builder for method chaining.
732 * @param namespace The journal serializer.
733 * @return The journal builder.
735 public Builder<E> withNamespace(JournalSerdes namespace) {
736 this.namespace = requireNonNull(namespace, "namespace cannot be null");
741 * Sets the maximum segment size in bytes, returning the builder for method chaining.
743 * The maximum segment size dictates when logs should roll over to new segments. As entries are written to a segment
744 * of the log, once the size of the segment surpasses the configured maximum segment size, the log will create a new
745 * segment and append new entries to that segment.
747 * By default, the maximum segment size is {@code 1024 * 1024 * 32}.
749 * @param maxSegmentSize The maximum segment size in bytes.
750 * @return The storage builder.
751 * @throws IllegalArgumentException If the {@code maxSegmentSize} is not positive
753 public Builder<E> withMaxSegmentSize(int maxSegmentSize) {
754 checkArgument(maxSegmentSize > JournalSegmentDescriptor.BYTES, "maxSegmentSize must be greater than " + JournalSegmentDescriptor.BYTES);
755 this.maxSegmentSize = maxSegmentSize;
760 * Sets the maximum entry size in bytes, returning the builder for method chaining.
762 * @param maxEntrySize the maximum entry size in bytes
763 * @return the storage builder
764 * @throws IllegalArgumentException if the {@code maxEntrySize} is not positive
766 public Builder<E> withMaxEntrySize(int maxEntrySize) {
767 checkArgument(maxEntrySize > 0, "maxEntrySize must be positive");
768 this.maxEntrySize = maxEntrySize;
773 * Sets the maximum number of allows entries per segment, returning the builder for method chaining.
775 * The maximum entry count dictates when logs should roll over to new segments. As entries are written to a segment
776 * of the log, if the entry count in that segment meets the configured maximum entry count, the log will create a
777 * new segment and append new entries to that segment.
779 * By default, the maximum entries per segment is {@code 1024 * 1024}.
781 * @param maxEntriesPerSegment The maximum number of entries allowed per segment.
782 * @return The storage builder.
783 * @throws IllegalArgumentException If the {@code maxEntriesPerSegment} not greater than the default max entries
785 * @deprecated since 3.0.2
788 public Builder<E> withMaxEntriesPerSegment(int maxEntriesPerSegment) {
789 checkArgument(maxEntriesPerSegment > 0, "max entries per segment must be positive");
790 checkArgument(maxEntriesPerSegment <= DEFAULT_MAX_ENTRIES_PER_SEGMENT,
791 "max entries per segment cannot be greater than " + DEFAULT_MAX_ENTRIES_PER_SEGMENT);
792 this.maxEntriesPerSegment = maxEntriesPerSegment;
797 * Sets the journal index density.
799 * The index density is the frequency at which the position of entries written to the journal will be recorded in an
800 * in-memory index for faster seeking.
802 * @param indexDensity the index density
803 * @return the journal builder
804 * @throws IllegalArgumentException if the density is not between 0 and 1
806 public Builder<E> withIndexDensity(double indexDensity) {
807 checkArgument(indexDensity > 0 && indexDensity < 1, "index density must be between 0 and 1");
808 this.indexDensity = indexDensity;
813 * Enables flushing buffers to disk when entries are committed to a segment, returning the builder for method
816 * When flush-on-commit is enabled, log entry buffers will be automatically flushed to disk each time an entry is
817 * committed in a given segment.
819 * @return The storage builder.
821 public Builder<E> withFlushOnCommit() {
822 return withFlushOnCommit(true);
826 * Sets whether to flush buffers to disk when entries are committed to a segment, returning the builder for method
829 * When flush-on-commit is enabled, log entry buffers will be automatically flushed to disk each time an entry is
830 * committed in a given segment.
832 * @param flushOnCommit Whether to flush buffers to disk when entries are committed to a segment.
833 * @return The storage builder.
835 public Builder<E> withFlushOnCommit(boolean flushOnCommit) {
836 this.flushOnCommit = flushOnCommit;
841 * Build the {@link SegmentedJournal}.
843 * @return A new {@link SegmentedJournal}.
845 public SegmentedJournal<E> build() {
846 return new SegmentedJournal<>(
853 maxEntriesPerSegment,