void appendEntry(LogEntryProto entry) { // SegmentedRaftLog does the segment creation/rolling work. Here we just // simply append the entry into the open segment. Preconditions.assertTrue(openSegment != null); openSegment.appendToOpenSegment(entry); }
/** * finalize the current open segment, and start a new open segment */ void rollOpenSegment(boolean createNewOpen) { Preconditions.assertTrue(openSegment != null && openSegment.numOfEntries() > 0); final long nextIndex = openSegment.getEndIndex() + 1; openSegment.close(); closedSegments.add(openSegment); clearOpenSegment(); if (createNewOpen) { addOpenSegment(nextIndex); } }
private boolean isSegmentFull(LogSegment segment, LogEntryProto entry) { if (segment.getTotalSize() >= segmentMaxSize) { return true; } else { final long entrySize = LogSegment.getEntrySize(entry); // if entry size is greater than the max segment size, write it directly // into the current segment return entrySize <= segmentMaxSize && segment.getTotalSize() + entrySize > segmentMaxSize; } }
@Override public int compareTo(Long l) { return (l >= getStartIndex() && l <= getEndIndex()) ? 0 : (this.getEndIndex() < l ? -1 : 1); }
private void checkLogSegment(LogSegment segment, long start, long end, boolean isOpen, long totalSize, long term) throws Exception { Assert.assertEquals(start, segment.getStartIndex()); Assert.assertEquals(end, segment.getEndIndex()); Assert.assertEquals(isOpen, segment.isOpen()); Assert.assertEquals(totalSize, segment.getTotalSize()); long offset = SegmentedRaftLog.HEADER_BYTES.length; for (long i = start; i <= end; i++) { LogSegment.LogRecord record = segment.getLogRecord(i); LogRecordWithEntry lre = segment.getEntryWithoutLoading(i); Assert.assertEquals(i, lre.getRecord().getTermIndex().getIndex()); Assert.assertEquals(term, lre.getRecord().getTermIndex().getTerm()); Assert.assertEquals(offset, record.getOffset()); LogEntryProto entry = lre.hasEntry() ? lre.getEntry() : segment.loadCache(lre.getRecord()); offset += getEntrySize(entry); } }
int segmentIndex = Collections.binarySearch(closedSegments, index); if (segmentIndex == -closedSegments.size() - 1) { if (openSegment != null && openSegment.getEndIndex() >= index) { final long oldEnd = openSegment.getEndIndex(); if (index == openSegment.getStartIndex()) { openSegment.truncate(index); Preconditions.assertTrue(!openSegment.isOpen()); SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(), oldEnd, true, openSegment.getTotalSize(), openSegment.getEndIndex()); closedSegments.add(openSegment); clearOpenSegment(); final long oldEnd = ts.getEndIndex(); List<SegmentFileInfo> list = new ArrayList<>(); ts.truncate(index); final int size = closedSegments.size(); for (int i = size - 1; i >= (ts.numOfEntries() == 0 ? segmentIndex : segmentIndex + 1); i-- ) { LogSegment s = closedSegments.remove(i); final long endOfS = i == segmentIndex ? oldEnd : s.getEndIndex(); s.clear(); list.add(new SegmentFileInfo(s.getStartIndex(), endOfS, false, 0, s.getEndIndex())); SegmentFileInfo t = ts.numOfEntries() == 0 ? null : new SegmentFileInfo(ts.getStartIndex(), oldEnd, false,
static LogSegment loadSegment(RaftStorage storage, File file, long start, long end, boolean isOpen, boolean keepEntryInCache, Consumer<LogEntryProto> logConsumer) throws IOException { final LogSegment segment = isOpen ? LogSegment.newOpenSegment(storage, start) : LogSegment.newCloseSegment(storage, start, end); readSegmentFile(file, start, end, isOpen, entry -> { segment.append(keepEntryInCache | isOpen, entry); if (logConsumer != null) { logConsumer.accept(entry); } }); // truncate padding if necessary if (file.length() > segment.getTotalSize()) { FileUtils.truncateFile(file, segment.getTotalSize()); } Preconditions.assertTrue(start == segment.getStartIndex()); if (!segment.records.isEmpty()) { Preconditions.assertTrue(start == segment.records.get(0).getTermIndex().getIndex()); } if (!isOpen) { Preconditions.assertTrue(segment.getEndIndex() == end); } return segment; }
@Test public void testTruncate() throws Exception { final long term = 1; final long start = 1000; LogSegment segment = LogSegment.newOpenSegment(null, start); for (int i = 0; i < 100; i++) { LogEntryProto entry = ProtoUtils.toLogEntryProto( new SimpleOperation("m" + i).getLogEntryContent(), term, i + start, clientId, callId); segment.appendToOpenSegment(entry); } // truncate an open segment (remove 1080~1099) long newSize = segment.getLogRecord(start + 80).getOffset(); segment.truncate(start + 80); Assert.assertEquals(80, segment.numOfEntries()); checkLogSegment(segment, start, start + 79, false, newSize, term); // truncate a closed segment (remove 1050~1079) newSize = segment.getLogRecord(start + 50).getOffset(); segment.truncate(start + 50); Assert.assertEquals(50, segment.numOfEntries()); checkLogSegment(segment, start, start + 49, false, newSize, term); // truncate all the remaining entries segment.truncate(start); Assert.assertEquals(0, segment.numOfEntries()); checkLogSegment(segment, start, start - 1, false, SegmentedRaftLog.HEADER_BYTES.length, term); }
if (!segment.isOpen() && segment.getEndIndex() <= localFlushedIndex) { break; for (; j <= safeIndex; j++) { LogSegment segment = segments.get(j); if (segment.getEndIndex() > lastAppliedIndex) { break; if (segment.hasCache()) { result.add(segment); for (int i = safeIndex; i >= j; i--) { LogSegment s = segments.get(i); if (s.getStartIndex() > lastAppliedIndex && s.hasCache()) { result.add(s); break; for (; j <= safeIndex; j++) { LogSegment s = segments.get(j); if (s.getEndIndex() >= minToRead) { break; if (s.hasCache()) { result.add(s); LogSegment s = segments.get(j); if (Arrays.stream(followerNextIndices).noneMatch(s::containsIndex) && !s.containsIndex(lastAppliedIndex) && s.hasCache()) { result.add(s);
@Test public void testAppendEntries() throws Exception { final long start = 1000; LogSegment segment = LogSegment.newOpenSegment(null, start); long size = SegmentedRaftLog.HEADER_BYTES.length; final long max = 8 * 1024 * 1024; checkLogSegment(segment, start, start - 1, true, size, 0); // append till full long term = 0; int i = 0; List<LogEntryProto> list = new ArrayList<>(); while (size < max) { SimpleOperation op = new SimpleOperation("m" + i); LogEntryProto entry = ProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i++ + start, clientId, callId); size += getEntrySize(entry); list.add(entry); } segment.appendToOpenSegment(list.toArray(new LogEntryProto[list.size()])); Assert.assertTrue(segment.getTotalSize() >= max); checkLogSegment(segment, start, i - 1 + start, true, size, term); }
@Override public void execute() throws IOException { IOUtils.cleanup(LOG, out); out = null; Preconditions.assertTrue(segmentToClose != null); File openFile = storage.getStorageDir() .getOpenLogFile(segmentToClose.getStartIndex()); LOG.debug("{} finalizing log segment {}", name, openFile); Preconditions.assertTrue(openFile.exists(), () -> name + ": File " + openFile + " does not exist, segmentToClose=" + segmentToClose.toDebugString()); if (segmentToClose.numOfEntries() > 0) { // finalize the current open segment File dstFile = storage.getStorageDir().getClosedLogFile( segmentToClose.getStartIndex(), segmentToClose.getEndIndex()); Preconditions.assertTrue(!dstFile.exists()); FileUtils.move(openFile, dstFile); } else { // delete the file of the empty segment FileUtils.deleteFile(openFile); } updateFlushedIndex(); }
private void validateAdding(LogSegment segment) { final LogSegment lastClosed = getLastClosedSegment(); if (lastClosed != null) { Preconditions.assertTrue(!lastClosed.isOpen()); Preconditions.assertTrue(lastClosed.getEndIndex() + 1 == segment.getStartIndex()); } }
private LogSegment prepareLogSegment(long start, long end, boolean isOpen) { LogSegment s = LogSegment.newOpenSegment(null, start); for (long i = start; i <= end; i++) { SimpleOperation m = new SimpleOperation("m" + i); LogEntryProto entry = ServerProtoUtils.toLogEntryProto(m.getLogEntryContent(), 0, i); s.appendToOpenSegment(entry); } if (!isOpen) { s.close(); } return s; }
private void getFromSegment(LogSegment segment, long startIndex, TermIndex[] entries, int offset, int size) { long endIndex = segment.getEndIndex(); endIndex = Math.min(endIndex, startIndex + size - 1); int index = offset; for (long i = startIndex; i <= endIndex; i++) { LogRecord r = segment.getLogRecord(i); entries[index++] = r == null ? null : r.getTermIndex(); } }
private void testLoadSegment(boolean loadInitial) throws Exception { // load an open segment File openSegmentFile = prepareLog(true, 0, 100, 0); RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR); LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0, INVALID_LOG_INDEX, true, loadInitial, null); checkLogSegment(openSegment, 0, 99, true, openSegmentFile.length(), 0); storage.close(); // for open segment we currently always keep log entries in the memory Assert.assertEquals(0, openSegment.getLoadingTimes()); // load a closed segment (1000-1099) File closedSegmentFile = prepareLog(false, 1000, 100, 1); LogSegment closedSegment = LogSegment.loadSegment(storage, closedSegmentFile, 1000, 1099, false, loadInitial, null); checkLogSegment(closedSegment, 1000, 1099, false, closedSegment.getTotalSize(), 1); Assert.assertEquals(loadInitial ? 0 : 1, closedSegment.getLoadingTimes()); }
static SegmentFileInfo deleteOpenSegment(LogSegment openSegment, Runnable clearOpenSegment) { final long oldEnd = openSegment.getEndIndex(); openSegment.clear(); final SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(), oldEnd, true, 0, openSegment.getEndIndex()); clearOpenSegment.run(); return info; } }
private List<LogSegment> prepareSegments(int numSegments, boolean[] cached, long start, long size) { Assert.assertEquals(numSegments, cached.length); List<LogSegment> segments = new ArrayList<>(numSegments); for (int i = 0; i < numSegments; i++) { LogSegment s = LogSegment.newCloseSegment(null, start, start + size - 1); if (cached[i]) { s = Mockito.spy(s); Mockito.when(s.hasCache()).thenReturn(true); } segments.add(s); start += size; } return segments; }
@Override public LogEntryProto get(long index) throws RaftLogIOException { checkLogState(); LogSegment segment; LogRecordWithEntry recordAndEntry; try (AutoCloseableLock readLock = readLock()) { segment = cache.getSegment(index); if (segment == null) { return null; } recordAndEntry = segment.getEntryWithoutLoading(index); if (recordAndEntry == null) { return null; } if (recordAndEntry.hasEntry()) { return recordAndEntry.getEntry(); } } // the entry is not in the segment's cache. Load the cache without holding // RaftLog's lock. checkAndEvictCache(); return segment.loadCache(recordAndEntry.getRecord()); }
@Test public void testAppendWithGap() throws Exception { LogSegment segment = LogSegment.newOpenSegment(null, 1000); SimpleOperation op = new SimpleOperation("m"); final SMLogEntryProto m = op.getLogEntryContent(); try { LogEntryProto entry = ProtoUtils.toLogEntryProto(m, 0, 1001, clientId, callId); segment.appendToOpenSegment(entry); Assert.fail("should fail since the entry's index needs to be 1000"); } catch (IllegalStateException e) { segment.appendToOpenSegment(entry); segment.appendToOpenSegment(entry); Assert.fail("should fail since the entry's index needs to be 1001"); } catch (IllegalStateException e) { segment.appendToOpenSegment(entries); Assert.fail("should fail since there is gap between entries"); } catch (IllegalStateException e) {