final int headerLength = SegmentedRaftLogFormat.getHeaderLength(); final int readLength = in.read(temp, 0, headerLength); Preconditions.assertTrue(readLength <= headerLength); final int matchLength = SegmentedRaftLogFormat.matchHeader(temp, 0, readLength); Preconditions.assertTrue(matchLength <= readLength); } else if (SegmentedRaftLogFormat.isTerminator(temp, matchLength, readLength - matchLength)) { + SegmentedRaftLogFormat.getHeaderLength() + ", read length=" + readLength + ", match length=" + matchLength + ", header in file=" + StringUtils.bytes2HexString(temp, 0, readLength) + ", expected header=" + SegmentedRaftLogFormat.applyHeaderTo(StringUtils::bytes2HexString));
static boolean isTerminator(byte[] bytes, int offset, int length) { return indexOfNonTerminator(bytes, offset, length) == -1; }
private LogSegment(RaftStorage storage, boolean isOpen, long start, long end) { this.storage = storage; this.isOpen = isOpen; this.startIndex = start; this.endIndex = end; totalSize = SegmentedRaftLogFormat.getHeaderLength(); hasEntryCache = isOpen; }
static void assertCorruptedLogHeader(RaftPeerId id, File openLogFile, int partialLength, MiniRaftCluster cluster, Logger LOG) throws Exception { Preconditions.assertTrue(partialLength < SegmentedRaftLogFormat.getHeaderLength()); try(final RandomAccessFile raf = new RandomAccessFile(openLogFile, "rw")) { SegmentedRaftLogFormat.applyHeaderTo(header -> { LOG.info("header = {}", StringUtils.bytes2HexString(header)); final byte[] corrupted = new byte[header.length]; System.arraycopy(header, 0, corrupted, 0, partialLength); LOG.info("corrupted = {}", StringUtils.bytes2HexString(corrupted)); raf.write(corrupted); return null; }); } final RaftServerImpl server = cluster.restartServer(id, false); server.getProxy().close(); }
private void create() throws IOException { fc.truncate(0); fc.position(0); preallocatedPos = 0; preallocate(); // preallocate file SegmentedRaftLogFormat.applyHeaderTo(CheckedConsumer.asCheckedFunction(out::write)); flush(); }
/** * @return The index of the first non-terminator if it exists. * Otherwise, return -1, i.e. all bytes are terminator. */ static int indexOfNonTerminator(byte[] bytes, int offset, int length) { for(int i = 0; i < length; i++) { if (!isTerminator(bytes[offset + i])) { return i; } } return -1; } }
static int matchHeader(byte[] bytes, int offset, int length) { Preconditions.assertTrue(length <= getHeaderLength()); for(int i = 0; i < length; i++) { if (bytes[offset + i] != Internal.HEADER_BYTES[i]) { return i; } } return length; }
@Test public void testSegmentedRaftLogFormatInternalHeader() throws Exception { testFailureCase("testSegmentedRaftLogFormatInternalHeader", () -> SegmentedRaftLogFormat.applyHeaderTo(header -> { LOG.info("header = " + new String(header, StandardCharsets.UTF_8)); header[0] += 1; // try changing the internal header LOG.info("header' = " + new String(header, StandardCharsets.UTF_8)); return null; }), IllegalStateException.class); // reset the header SegmentedRaftLogFormat.applyHeaderTo(header -> { LOG.info("header' = " + new String(header, StandardCharsets.UTF_8)); header[0] -= 1; // try changing the internal header LOG.info("header'' = " + new String(header, StandardCharsets.UTF_8)); return null; }); } }
void verifyTerminator() throws IOException { // The end of the log should contain 0x00 bytes. // If it contains other bytes, the log itself may be corrupt. limiter.clearLimit(); int numRead = -1, idx = 0; while (true) { try { numRead = in.read(temp); if (numRead == -1) { return; } for (idx = 0; idx < numRead; idx++) { if (!SegmentedRaftLogFormat.isTerminator(temp[idx])) { throw new IOException("Read extra bytes after the terminator!"); } } } finally { // After reading each group of bytes, we reposition the mark one // byte before the next group. Similarly, if there is an error, we // want to reposition the mark one byte before the error if (numRead != -1) { in.reset(); IOUtils.skipFully(in, idx); in.mark(temp.length + 1); IOUtils.skipFully(in, 1); } } } }
@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); }
if (SegmentedRaftLogFormat.isTerminator(nextByte)) { verifyTerminator(); return null;
/** * Keep appending and check if pre-allocation is correct */ @Test public void testPreallocationAndAppend() throws Exception { final SizeInBytes max = SizeInBytes.valueOf(2, TraditionalBinaryPrefix.MEGA); RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR); final File file = storage.getStorageDir().getOpenLogFile(0); final byte[] content = new byte[1024]; Arrays.fill(content, (byte) 1); SimpleOperation op = new SimpleOperation(new String(content)); LogEntryProto entry = ServerProtoUtils.toLogEntryProto(op.getLogEntryContent(), 0, 0); final long entrySize = LogSegment.getEntrySize(entry); long totalSize = SegmentedRaftLogFormat.getHeaderLength(); long preallocated = 16 * 1024; try (LogOutputStream out = new LogOutputStream(file, false, max.getSize(), 16 * 1024, 10 * 1024)) { Assert.assertEquals(preallocated, file.length()); while (totalSize + entrySize < max.getSize()) { totalSize += entrySize; out.write(entry); if (totalSize > preallocated) { Assert.assertEquals("totalSize==" + totalSize, preallocated + 16 * 1024, file.length()); preallocated += 16 * 1024; } } } Assert.assertEquals(totalSize, file.length()); }
/** * Test basic functionality: write several log entries, then read */ @Test public void testReadWriteLog() throws IOException { final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR); File openSegment = storage.getStorageDir().getOpenLogFile(0); long size = SegmentedRaftLogFormat.getHeaderLength(); final LogEntryProto[] entries = new LogEntryProto[100]; try (LogOutputStream out = new LogOutputStream(openSegment, false, segmentMaxSize, preallocatedSize, bufferSize)) { size += writeMessages(entries, out); } finally { storage.close(); } Assert.assertEquals(size, openSegment.length()); LogEntryProto[] readEntries = readLog(openSegment, 0, RaftServerConstants.INVALID_LOG_INDEX, true); Assert.assertArrayEquals(entries, readEntries); }
void runTestRestartWithCorruptedLogHeader(MiniRaftCluster cluster) throws Exception { RaftTestUtil.waitForLeader(cluster); for(RaftServerImpl impl : cluster.iterateServerImpls()) { JavaUtils.attempt(() -> getOpenLogFile(impl), 10, TimeDuration.valueOf(100, TimeUnit.MILLISECONDS), impl.getId() + ": wait for log file creation", LOG); } // shutdown all servers cluster.getServers().forEach(RaftServerProxy::close); for(RaftServerImpl impl : cluster.iterateServerImpls()) { final File openLogFile = JavaUtils.attempt(() -> getOpenLogFile(impl), 10, 100, impl.getId() + "-getOpenLogFile", LOG); for(int i = 0; i < SegmentedRaftLogFormat.getHeaderLength(); i++) { assertCorruptedLogHeader(impl.getId(), openLogFile, i, cluster, LOG); Assert.assertTrue(getOpenLogFiles(impl).isEmpty()); } } }
/** * Simulate the scenario that the peer is shutdown without truncating * log segment file padding. Make sure the reader can correctly handle this. */ @Test public void testReadWithPadding() throws IOException { final RaftStorage storage = new RaftStorage(storageDir, StartupOption.REGULAR); File openSegment = storage.getStorageDir().getOpenLogFile(0); long size = SegmentedRaftLogFormat.getHeaderLength(); LogEntryProto[] entries = new LogEntryProto[100]; LogOutputStream out = new LogOutputStream(openSegment, false, segmentMaxSize, preallocatedSize, bufferSize); size += writeMessages(entries, out); out.flush(); // make sure the file contains padding Assert.assertEquals( RaftServerConfigKeys.Log.PREALLOCATED_SIZE_DEFAULT.getSize(), openSegment.length()); // check if the reader can correctly read the log file LogEntryProto[] readEntries = readLog(openSegment, 0, RaftServerConstants.INVALID_LOG_INDEX, true); Assert.assertArrayEquals(entries, readEntries); out.close(); Assert.assertEquals(size, openSegment.length()); }
size + SegmentedRaftLogFormat.getHeaderLength()); try (LogInputStream in = new LogInputStream(file, 0, INVALID_LOG_INDEX, true)) {
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); } }
@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 = ServerProtoUtils.toLogEntryProto( new SimpleOperation("m" + i).getLogEntryContent(), term, i + start); 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, SegmentedRaftLogFormat.getHeaderLength(), term); }