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; } }
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; } }
} else if (file.length() > segment.getTotalSize()) { FileUtils.truncateFile(file, segment.getTotalSize());
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; }
Preconditions.assertTrue(!openSegment.isOpen()); SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(), oldEnd, true, openSegment.getTotalSize(), openSegment.getEndIndex()); closedSegments.add(openSegment); ts.getTotalSize(), ts.getEndIndex()); return new TruncationSegments(t, list);
Preconditions.assertTrue(!openSegment.isOpen()); final SegmentFileInfo info = new SegmentFileInfo(openSegment.getStartIndex(), oldEnd, true, openSegment.getTotalSize(), openSegment.getEndIndex()); segments.add(openSegment); clearOpenSegment.run(); new SegmentFileInfo(ts.getStartIndex(), oldEnd, false, ts.getTotalSize(), ts.getEndIndex()); return new TruncationSegments(t, list);
@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); }
@Test public void testAppendEntries() throws Exception { final long start = 1000; LogSegment segment = LogSegment.newOpenSegment(null, start); long size = SegmentedRaftLogFormat.getHeaderLength(); 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 = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), term, i++ + start); 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); }
private void testLoadSegment(boolean loadInitial, boolean isLastEntryPartiallyWritten) throws Exception { // load an open segment final File openSegmentFile = prepareLog(true, 0, 100, 0, isLastEntryPartiallyWritten); RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR); LogSegment openSegment = LogSegment.loadSegment(storage, openSegmentFile, 0, INVALID_LOG_INDEX, true, loadInitial, null); final int delta = isLastEntryPartiallyWritten? 1: 0; checkLogSegment(openSegment, 0, 99 - delta, 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) final File closedSegmentFile = prepareLog(false, 1000, 100, 1, false); 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()); }
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()); }
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); } }
static 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 = SegmentedRaftLogFormat.getHeaderLength(); 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); } }