void loadSegment(LogPathAndIndex pi, boolean isOpen, boolean keepEntryInCache, Consumer<LogEntryProto> logConsumer) throws IOException { LogSegment logSegment = LogSegment.loadSegment(storage, pi.path.toFile(), pi.startIndex, pi.endIndex, isOpen, keepEntryInCache, logConsumer); addSegment(logSegment); }
boolean shouldEvict() { return getCachedSegmentNum() > maxCachedSegments; }
private void loadLogSegments(long lastIndexInSnapshot, Consumer<LogEntryProto> logConsumer) throws IOException { try(AutoCloseableLock writeLock = writeLock()) { List<LogPathAndIndex> paths = storage.getStorageDir().getLogSegmentFiles(); int i = 0; for (LogPathAndIndex pi : paths) { // During the initial loading, we can only confirm the committed // index based on the snapshot. This means if a log segment is not kept // in cache after the initial loading, later we have to load its content // again for updating the state machine. // TODO we should let raft peer persist its committed index periodically // so that during the initial loading we can apply part of the log // entries to the state machine boolean keepEntryInCache = (paths.size() - i++) <= cache.getMaxCachedSegments(); cache.loadSegment(pi, keepEntryInCache, logConsumer); } // if the largest index is smaller than the last index in snapshot, we do // not load the log to avoid holes between log segments. This may happen // when the local I/O worker is too slow to persist log (slower than // committing the log and taking snapshot) if (!cache.isEmpty() && cache.getEndIndex() < lastIndexInSnapshot) { LOG.warn("End log index {} is smaller than last index in snapshot {}", cache.getEndIndex(), lastIndexInSnapshot); cache.clear(); // TODO purge all segment files } } }
/** * @param startIndex inclusive * @param endIndex exclusive */ TermIndex[] getTermIndices(final long startIndex, final long endIndex) { if (startIndex < 0 || startIndex < getStartIndex()) { throw new IndexOutOfBoundsException("startIndex = " + startIndex + ", log cache starts from index " + getStartIndex()); } if (startIndex > endIndex) { throw new IndexOutOfBoundsException("startIndex(" + startIndex + ") > endIndex(" + endIndex + ")"); } final long realEnd = Math.min(getEndIndex() + 1, endIndex); if (startIndex >= realEnd) { return TermIndex.EMPTY_TERMINDEX_ARRAY; } return closedSegments.getTermIndex(startIndex, realEnd, openSegment); }
if (startIndex < 0 || startIndex < getStartIndex()) { throw new IndexOutOfBoundsException("startIndex = " + startIndex + ", log cache starts from index " + getStartIndex()); + ") > endIndex(" + endIndex + ")"); final long realEnd = Math.min(getEndIndex() + 1, endIndex); if (startIndex >= realEnd) { return TermIndex.EMPTY_TERMINDEX_ARRAY; int segmentIndex = Collections.binarySearch(closedSegments, startIndex); if (segmentIndex < 0) { getFromSegment(openSegment, startIndex, entries, 0, entries.length); } else { long index = startIndex; int numberFromSegment = Math.toIntExact( Math.min(realEnd - index, s.getEndIndex() - index + 1)); getFromSegment(s, index, entries, Math.toIntExact(index - startIndex), numberFromSegment); index += numberFromSegment; getFromSegment(openSegment, index, entries, Math.toIntExact(index - startIndex), Math.toIntExact(realEnd - index));
for (int i = 0; i < 5; i++) { // 5 closed segments LogSegment s = prepareLogSegment(start, start + 99, false); cache.addSegment(s); start += 100; cache.addSegment(s); long end = cache.getEndIndex(); Assert.assertEquals(599, end); int numOfSegments = 6; TruncationSegments ts = cache.truncate(end + 1); checkCache(0, end, 100); TruncationSegments ts = cache.truncate(200); checkCache(0, 199, 100); Assert.assertEquals(1, ts.toDelete.length); cache.addSegment(newOpen); ts = cache.truncate(200); checkCache(0, 199, 100); Assert.assertEquals(1, ts.toDelete.length); cache.addSegment(newOpen); ts = cache.truncate(220); checkCache(0, 219, 100); Assert.assertNull(cache.getOpenSegment()); Assert.assertEquals(0, ts.toDelete.length); Assert.assertTrue(ts.toTruncate.isOpen);
final LogSegment currentOpenSegment = cache.getOpenSegment(); if (currentOpenSegment == null) { cache.addOpenSegment(entry.getIndex()); fileLogWorker.startLogSegment(entry.getIndex()); } else if (isSegmentFull(currentOpenSegment, entry)) { cache.rollOpenSegment(true); fileLogWorker.rollLogSegment(currentOpenSegment); checkAndEvictCache(); "open segment's term %s is larger than the new entry's term %s", currentTerm, entry.getTerm()); cache.rollOpenSegment(true); fileLogWorker.rollLogSegment(currentOpenSegment); checkAndEvictCache(); cache.appendEntry(entry); return writeFuture; } catch (Throwable throwable) {
private void testIterator(long startIndex) throws IOException { Iterator<TermIndex> iterator = cache.iterator(startIndex); TermIndex prev = null; while (iterator.hasNext()) { TermIndex termIndex = iterator.next(); Assert.assertEquals(cache.getLogRecord(termIndex.getIndex()).getTermIndex(), termIndex); if (prev != null) { Assert.assertEquals(prev.getIndex() + 1, termIndex.getIndex()); } prev = termIndex; } if (startIndex <= cache.getEndIndex()) { Assert.assertNotNull(prev); Assert.assertEquals(cache.getEndIndex(), prev.getIndex()); } }
private void checkCache(long start, long end, int segmentSize) throws IOException { Assert.assertEquals(start, cache.getStartIndex()); Assert.assertEquals(end, cache.getEndIndex()); for (long index = start; index <= end; index++) { LogEntryProto entry = cache.getSegment(index).getEntryWithoutLoading(index).getEntry(); Assert.assertEquals(index, entry.getIndex()); } long[] offsets = new long[]{start, start + 1, start + (end - start) / 2, end - 1, end}; for (long offset : offsets) { checkCacheEntries(offset, (int) (end - offset + 1), end); checkCacheEntries(offset, 1, end); checkCacheEntries(offset, 20, end); checkCacheEntries(offset, segmentSize, end); checkCacheEntries(offset, segmentSize - 1, end); } }
@Test public void testAppendEntry() throws Exception { LogSegment closedSegment = prepareLogSegment(0, 99, false); cache.addSegment(closedSegment); final SimpleOperation m = new SimpleOperation("m"); try { LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, 0); cache.appendEntry(entry); Assert.fail("the open segment is null"); } catch (IllegalStateException ignored) { } LogSegment openSegment = prepareLogSegment(100, 100, true); cache.addSegment(openSegment); for (long index = 101; index < 200; index++) { LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, index); cache.appendEntry(entry); } Assert.assertNotNull(cache.getOpenSegment()); checkCache(0, 199, 100); }
@Test public void testAddSegments() throws Exception { LogSegment s1 = prepareLogSegment(1, 100, false); cache.addSegment(s1); checkCache(1, 100, 100); cache.addSegment(s); Assert.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { cache.addSegment(s2); checkCache(1, 200, 100); cache.addSegment(s); Assert.fail("should fail since there is still an open segment in cache"); } catch (IllegalStateException ignored) { cache.rollOpenSegment(false); checkCache(1, 200, 100); cache.addSegment(s); Assert.fail("should fail since there is gap between two segments"); } catch (IllegalStateException ignored) { cache.addSegment(s3); Assert.assertNotNull(cache.getOpenSegment()); checkCache(1, 300, 100); cache.rollOpenSegment(true); Assert.assertNotNull(cache.getOpenSegment());
@Override protected void openImpl(long lastIndexInSnapshot, Consumer<LogEntryProto> consumer) throws IOException { loadLogSegments(lastIndexInSnapshot, consumer); File openSegmentFile = null; LogSegment openSegment = cache.getOpenSegment(); if (openSegment != null) { openSegmentFile = storage.getStorageDir() .getOpenLogFile(openSegment.getStartIndex()); } fileLogWorker.start(Math.max(cache.getEndIndex(), lastIndexInSnapshot), openSegmentFile); }
@Test public void testIterator() throws Exception { long start = 0; for (int i = 0; i < 2; i++) { // 2 closed segments LogSegment s = prepareLogSegment(start, start + 99, false); cache.addSegment(s); start += 100; } // add another open segment LogSegment s = prepareLogSegment(start, start + 99, true); cache.addSegment(s); for (long startIndex = 0; startIndex < 300; startIndex += 50) { testIterator(startIndex); } testIterator(299); Iterator<TermIndex> iterator = cache.iterator(300); Assert.assertFalse(iterator.hasNext()); } }
LogRecord getLogRecord(long index) { LogSegment segment = getSegment(index); return segment == null ? null : segment.getLogRecord(index); }
@Override public TermIndex[] getEntries(long startIndex, long endIndex) { checkLogState(); try(AutoCloseableLock readLock = readLock()) { return cache.getTermIndices(startIndex, endIndex); } }
@Before public void setup() { cache = new RaftLogCache(null, null, prop); }
@Override public TermIndex getTermIndex(long index) { checkLogState(); try(AutoCloseableLock readLock = readLock()) { LogRecord record = cache.getLogRecord(index); return record != null ? record.getTermIndex() : null; } }
@Override public long getStartIndex() { return cache.getStartIndex(); }
for (int i = 0; i < 5; i++) { // 5 closed segments LogSegment s = prepareLogSegment(start, start + 99, false); cache.addSegment(s); start += 100; cache.addSegment(s); long end = cache.getEndIndex(); Assert.assertEquals(599, end); int numOfSegments = 6; TruncationSegments ts = cache.truncate(end + 1); checkCache(0, end, 100); TruncationSegments ts = cache.truncate(200); checkCache(0, 199, 100); Assert.assertEquals(1, ts.toDelete.length); cache.addSegment(newOpen); ts = cache.truncate(200); checkCache(0, 199, 100); Assert.assertEquals(1, ts.toDelete.length); cache.addSegment(newOpen); ts = cache.truncate(220); checkCache(0, 219, 100); Assert.assertNull(cache.getOpenSegment()); Assert.assertEquals(0, ts.toDelete.length); Assert.assertTrue(ts.toTruncate.isOpen);
final LogSegment currentOpenSegment = cache.getOpenSegment(); if (currentOpenSegment == null) { cache.addOpenSegment(entry.getIndex()); fileLogWorker.startLogSegment(entry.getIndex()); } else if (isSegmentFull(currentOpenSegment, entry)) { cache.rollOpenSegment(true); fileLogWorker.rollLogSegment(currentOpenSegment); checkAndEvictCache(); "open segment's term %s is larger than the new entry's term %s", currentTerm, entry.getTerm()); cache.rollOpenSegment(true); fileLogWorker.rollLogSegment(currentOpenSegment); checkAndEvictCache(); if (stateMachineCachingEnabled) { cache.appendEntry(ServerProtoUtils.removeStateMachineData(entry)); } else { cache.appendEntry(entry);