From 49dc1e00eda61d278700747096826763e514d4bd Mon Sep 17 00:00:00 2001 From: Ivan Kelly Date: Mon, 28 Sep 2020 20:51:08 +0200 Subject: [PATCH 01/15] Direct I/O entry log support The implementation uses JNI to do direct I/O to files via posix syscalls. Fallocate is used if running on linux, otherwise this is skipped (at the cost of more filesystem operates during writing). There are two calls to write, writeAt and writeDelimited. I expect writeAt to be used for the entrylog headers, which entries will go through writeDelimited. In both cases, the calls may return before the syscalls occur. #flush() needs to be called to ensure things are actually written. The entry log format isn't much changed from what is used by the existing entrylogger. The biggest difference is the padding. Direct I/O must write in aligned blocked. The size of the alignment varies by machine configuration, but 4K is a safe bet on most. As it is unlikely that entry data will land exactly on the alignment boundary, we need to add padding to writes. The existing entry logger has been changed to take this padding into account. When read as a signed int/long/byte the padding will aways parse to a negative value, which distinguishes it from valid entry data (the entry size will always be positive) and also from preallocated space (which is always 0). Another difference in the format is that the header is now 4K rather than 1K. Again, this is to allow aligned rights. No changes are necessary to allow the existing entry logger to deal with the header change, as we create a dummy entry in the extra header space that the existing entry logger already knows to ignore. To enable, set dbStorage_directIOEntryLogger=true in the configuration. (cherry picked from commit 09a1c81b6894d4398c5a59e54cc18784550badb5) --- .../bookie/AbstractLogCompactor.java | 7 +- .../bookkeeper/bookie/DefaultEntryLogger.java | 17 +- .../bookkeeper/bookie/EntryLocation.java | 10 + .../bookie/storage/EntryLogIds.java | 33 + .../bookie/storage/EntryLogIdsImpl.java | 148 +++++ .../DirectCompactionEntryLog.java | 281 ++++++++ .../directentrylogger/DirectEntryLogger.java | 507 +++++++++++++++ .../DirectEntryLoggerStats.java | 174 +++++ .../directentrylogger/DirectReader.java | 344 ++++++++++ .../directentrylogger/DirectWriter.java | 322 +++++++++ .../storage/directentrylogger/Events.java | 145 +++++ .../storage/directentrylogger/Header.java | 105 +++ .../directentrylogger/LogMetadata.java | 192 ++++++ .../storage/directentrylogger/LogReader.java | 80 +++ .../directentrylogger/LogReaderScan.java | 61 ++ .../storage/directentrylogger/LogWriter.java | 81 +++ .../directentrylogger/WriterWithMetadata.java | 79 +++ .../bookie/storage/ldb/DbLedgerStorage.java | 110 +++- .../ldb/SingleDirectoryDbLedgerStorage.java | 8 +- .../bookie/GarbageCollectorThreadTest.java | 9 + .../bookkeeper/bookie/MockLedgerStorage.java | 18 +- .../bookie/storage/EntryLogTestUtils.java | 42 +- .../bookie/storage/MockEntryLogIds.java | 34 + .../bookie/storage/TestEntryLogIds.java | 246 +++++++ .../TestDirectEntryLogger.java | 515 +++++++++++++++ .../TestDirectEntryLoggerCompat.java | 329 ++++++++++ .../directentrylogger/TestDirectReader.java | 509 +++++++++++++++ .../directentrylogger/TestDirectWriter.java | 333 ++++++++++ .../TestTransactionalEntryLogCompactor.java | 615 ++++++++++++++++++ .../ldb/DbLedgerStorageWriteCacheTest.java | 8 +- 30 files changed, 5333 insertions(+), 29 deletions(-) create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIds.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Header.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReader.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReaderScan.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java create mode 100644 bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/WriterWithMetadata.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/MockEntryLogIds.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectWriter.java create mode 100644 bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestTransactionalEntryLogCompactor.java diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java index f5ae90b3362..ff35f570ef9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/AbstractLogCompactor.java @@ -33,7 +33,10 @@ public abstract class AbstractLogCompactor { protected final ServerConfiguration conf; protected final Throttler throttler; - interface LogRemovalListener { + /** + * LogRemovalListener. + */ + public interface LogRemovalListener { void removeEntryLog(long logToRemove); } @@ -71,7 +74,7 @@ public static class Throttler { } // acquire. if bybytes: bytes of this entry; if byentries: 1. - void acquire(int permits) { + public void acquire(int permits) { rateLimiter.acquire(this.isThrottleByBytes ? permits : 1); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java index 1beae25370d..bc9513c4351 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java @@ -1005,11 +1005,17 @@ public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOExce return; } long offset = pos; - pos += 4; + int entrySize = headerBuffer.readInt(); + if (entrySize <= 0) { // hitting padding + pos++; + headerBuffer.clear(); + continue; + } long ledgerId = headerBuffer.readLong(); headerBuffer.clear(); + pos += 4; if (ledgerId == INVALID_LID || !scanner.accept(ledgerId)) { // skip this entry pos += entrySize; @@ -1017,11 +1023,6 @@ public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOExce } // read the entry data.clear(); - if (entrySize <= 0) { - LOG.warn("bad read for ledger entry from entryLog {}@{} (entry size {})", - entryLogId, pos, entrySize); - return; - } data.capacity(entrySize); int rc = readFromLogChannel(entryLogId, bc, data, pos); if (rc != entrySize) { @@ -1086,7 +1087,9 @@ EntryLogMetadata extractEntryLogMetadataFromIndex(long entryLogId) throws IOExce bc.read(sizeBuffer.get(), offset); int ledgersMapSize = sizeBuffer.get().readInt(); - + if (ledgersMapSize <= 0) { + break; + } // Read the index into a buffer ledgersMap.clear(); bc.read(ledgersMap, offset + 4, ledgersMapSize); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLocation.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLocation.java index be5eb7fbee8..f43521d9e95 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLocation.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLocation.java @@ -46,4 +46,14 @@ public long getEntry() { public long getLocation() { return location; } + + @Override + public String toString() { + return new StringBuilder().append("EntryLocation{") + .append("ledger=").append(ledger) + .append(",entry=").append(entry) + .append(",locationLog=").append(location >> 32 & 0xFFFFFFFF) + .append(",locationOffset=").append((int) (location & 0xFFFFFFFF)) + .append("}").toString(); + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIds.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIds.java new file mode 100644 index 00000000000..946c47bd03c --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIds.java @@ -0,0 +1,33 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage; + +import java.io.IOException; + +/** + * Generate unique entry log ids. + */ +public interface EntryLogIds { + /** + * Get the next available entry log ID. + */ + int nextId() throws IOException; +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java new file mode 100644 index 00000000000..f2b070088a5 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java @@ -0,0 +1,148 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.bookie.storage.directentrylogger.Events; +import org.apache.bookkeeper.slogger.Slogger; + +/** + * EntryLogIdsImpl. + */ +public class EntryLogIdsImpl implements EntryLogIds { + public static final Pattern FILE_PATTERN = Pattern.compile("^([0-9a-fA-F]+)\\.log$"); + public static final Pattern COMPACTED_FILE_PATTERN = + Pattern.compile("^([0-9a-fA-F]+)\\.log\\.([0-9a-fA-F]+)\\.compacted$"); + + private final LedgerDirsManager ledgerDirsManager; + private final Slogger slog; + private int nextId; + private int maxId; + + public EntryLogIdsImpl(LedgerDirsManager ledgerDirsManager, + Slogger slog) throws IOException { + this.ledgerDirsManager = ledgerDirsManager; + this.slog = slog; + findLargestGap(); + } + + @Override + public int nextId() throws IOException { + while (true) { + synchronized (this) { + int current = nextId; + nextId++; + if (nextId == maxId) { + findLargestGap(); + } else { + return current; + } + } + } + } + + private void findLargestGap() throws IOException { + long start = System.nanoTime(); + List currentIds = new ArrayList(); + + for (File ledgerDir : ledgerDirsManager.getAllLedgerDirs()) { + currentIds.addAll(logIdsInDirectory(ledgerDir)); + currentIds.addAll(compactedLogIdsInDirectory(ledgerDir)); + } + + int[] gap = findLargestGap(currentIds); + nextId = gap[0]; + maxId = gap[1]; + slog.kv("dirs", ledgerDirsManager.getAllLedgerDirs()) + .kv("nextId", nextId) + .kv("maxId", maxId) + .kv("durationMs", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)) + .info(Events.ENTRYLOG_IDS_CANDIDATES_SELECTED); + } + + /** + * O(nlogn) algorithm to find largest contiguous gap between + * integers in a passed list. n should be relatively small. + * Entry logs should be about 1GB in size, so even if the node + * stores a PB, there should be only 1000000 entry logs. + */ + static int[] findLargestGap(List currentIds) { + if (currentIds.isEmpty()) { + return new int[] { 0, Integer.MAX_VALUE }; + } + + Collections.sort(currentIds); + + int nextIdCandidate = 0; + int maxIdCandidate = currentIds.get(0); + int maxGap = maxIdCandidate - nextIdCandidate; + for (int i = 0; i < currentIds.size(); i++) { + int gapStart = currentIds.get(i) + 1; + int j = i + 1; + int gapEnd = Integer.MAX_VALUE; + if (j < currentIds.size()) { + gapEnd = currentIds.get(j); + } + int gapSize = gapEnd - gapStart; + if (gapSize > maxGap) { + maxGap = gapSize; + nextIdCandidate = gapStart; + maxIdCandidate = gapEnd; + } + } + return new int[] { nextIdCandidate, maxIdCandidate }; + } + + public static List logIdsInDirectory(File directory) { + List ids = new ArrayList<>(); + File[] files = directory.listFiles(); + for (File f : files) { + Matcher m = FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int logId = Integer.parseUnsignedInt(m.group(1), 16); + ids.add(logId); + } + } + return ids; + } + + private static List compactedLogIdsInDirectory(File directory) { + List ids = new ArrayList<>(); + File[] files = directory.listFiles(); + for (File f : files) { + Matcher m = COMPACTED_FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int logId = Integer.parseUnsignedInt(m.group(1), 16); + ids.add(logId); + } + } + return ids; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java new file mode 100644 index 00000000000..cce4cc63e99 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java @@ -0,0 +1,281 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTED_SUFFIX; +import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; + +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.concurrent.ExecutorService; + +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.storage.CompactionEntryLog; +import org.apache.bookkeeper.bookie.storage.EntryLogScanner; +import org.apache.bookkeeper.common.util.nativeio.NativeIO; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.OpStatsLogger; + +/** + * DirectCompactionEntryLog. + */ +public abstract class DirectCompactionEntryLog implements CompactionEntryLog { + protected final int srcLogId; + protected final int dstLogId; + protected final Slogger slog; + + protected final File compactingFile; + protected final File compactedFile; + protected final File completeFile; + + static CompactionEntryLog newLog(int srcLogId, + int dstLogId, + File ledgerDir, + long maxFileSize, + ExecutorService writeExecutor, + BufferPool writeBuffers, + NativeIO nativeIO, + ByteBufAllocator allocator, + Slogger slog) throws IOException { + return new WritingDirectCompactionEntryLog( + srcLogId, dstLogId, ledgerDir, maxFileSize, + writeExecutor, writeBuffers, nativeIO, allocator, slog); + } + + static CompactionEntryLog recoverLog(int srcLogId, + int dstLogId, + File ledgerDir, + int readBufferSize, + int maxSaneEntrySize, + NativeIO nativeIO, + ByteBufAllocator allocator, + OpStatsLogger readBlockStats, + Slogger slog) { + return new RecoveredDirectCompactionEntryLog(srcLogId, dstLogId, ledgerDir, readBufferSize, + maxSaneEntrySize, nativeIO, allocator, readBlockStats, slog); + } + + private DirectCompactionEntryLog(int srcLogId, + int dstLogId, + File ledgerDir, + Slogger slog) { + compactingFile = compactingFile(ledgerDir, dstLogId); + compactedFile = compactedFile(ledgerDir, dstLogId, srcLogId); + completeFile = DirectEntryLogger.logFile(ledgerDir, dstLogId); + + this.srcLogId = srcLogId; + this.dstLogId = dstLogId; + + this.slog = slog.kv("dstLogId", dstLogId).kv("srcLogId", srcLogId).ctx(); + } + + @Override + public void abort() { + try { + Files.deleteIfExists(compactingFile.toPath()); + } catch (IOException ioe) { + slog.kv("compactingFile", compactingFile).warn(Events.COMPACTION_ABORT_EXCEPTION, ioe); + } + + try { + Files.deleteIfExists(compactedFile.toPath()); + } catch (IOException ioe) { + slog.kv("compactedFile", compactedFile).warn(Events.COMPACTION_ABORT_EXCEPTION, ioe); + } + } + + + @Override + public void makeAvailable() throws IOException { + idempotentLink(compactedFile, completeFile); + slog.kv("compactedFile", compactedFile).kv("completeFile", completeFile) + .info(Events.COMPACTION_MAKE_AVAILABLE); + } + + private static void idempotentLink(File src, File dst) throws IOException { + if (!src.exists()) { + throw new IOException(exMsg("src doesn't exist, aborting link") + .kv("src", src).kv("dst", dst).toString()); + } + if (!dst.exists()) { + Files.createLink(dst.toPath(), src.toPath()); + } else if (!Files.isSameFile(src.toPath(), dst.toPath())) { + throw new IOException(exMsg("dst exists, but doesn't match src") + .kv("src", src) + .kv("dst", dst).toString()); + } // else src and dst point to the same inode so we have nothing to do + } + + @Override + public void finalizeAndCleanup() { + try { + Files.deleteIfExists(compactingFile.toPath()); + } catch (IOException ioe) { + slog.kv("compactingFile", compactingFile).warn(Events.COMPACTION_DELETE_FAILURE, ioe); + } + + try { + Files.deleteIfExists(compactedFile.toPath()); + } catch (IOException ioe) { + slog.kv("compactedFile", compactedFile).warn(Events.COMPACTION_DELETE_FAILURE, ioe); + } + slog.info(Events.COMPACTION_COMPLETE); + } + + @Override + public long getDstLogId() { + return dstLogId; + } + + @Override + public long getSrcLogId() { + return srcLogId; + } + + private static class RecoveredDirectCompactionEntryLog extends DirectCompactionEntryLog { + private final ByteBufAllocator allocator; + private final NativeIO nativeIO; + private final int readBufferSize; + private final int maxSaneEntrySize; + private final OpStatsLogger readBlockStats; + + RecoveredDirectCompactionEntryLog(int srcLogId, + int dstLogId, + File ledgerDir, + int readBufferSize, + int maxSaneEntrySize, + NativeIO nativeIO, + ByteBufAllocator allocator, + OpStatsLogger readBlockStats, + Slogger slog) { + super(srcLogId, dstLogId, ledgerDir, slog); + this.allocator = allocator; + this.nativeIO = nativeIO; + this.readBufferSize = readBufferSize; + this.maxSaneEntrySize = maxSaneEntrySize; + this.readBlockStats = readBlockStats; + + this.slog.info(Events.COMPACTION_LOG_RECOVERED); + } + + private IllegalStateException illegalOpException() { + return new IllegalStateException(exMsg("Invalid operation for recovered log") + .kv("srcLogId", srcLogId) + .kv("dstLogId", dstLogId) + .kv("compactingFile", compactingFile) + .kv("compactedFile", compactedFile) + .kv("completeFile", completeFile).toString()); + } + + @Override + public long addEntry(long ledgerId, ByteBuf entry) throws IOException { + throw illegalOpException(); + } + + @Override + public void flush() throws IOException { + throw illegalOpException(); + } + + @Override + public void markCompacted() throws IOException { + throw illegalOpException(); + } + + @Override + public void scan(EntryLogScanner scanner) throws IOException { + try (LogReader reader = new DirectReader(dstLogId, compactedFile.toString(), allocator, nativeIO, + readBufferSize, maxSaneEntrySize, readBlockStats)) { + LogReaderScan.scan(reader, scanner); + } + } + } + + private static class WritingDirectCompactionEntryLog extends DirectCompactionEntryLog { + private final WriterWithMetadata writer; + + WritingDirectCompactionEntryLog(int srcLogId, + int dstLogId, + File ledgerDir, + long maxFileSize, + ExecutorService writeExecutor, + BufferPool writeBuffers, + NativeIO nativeIO, + ByteBufAllocator allocator, + Slogger slog) throws IOException { + super(srcLogId, dstLogId, ledgerDir, slog); + + this.writer = new WriterWithMetadata( + new DirectWriter(dstLogId, compactingFile.toString(), maxFileSize, + writeExecutor, writeBuffers, nativeIO, slog), + new EntryLogMetadata(dstLogId), + allocator); + + this.slog.info(Events.COMPACTION_LOG_CREATED); + } + + @Override + public long addEntry(long ledgerId, ByteBuf entry) throws IOException { + return writer.addEntry(ledgerId, entry); + } + + @Override + public void flush() throws IOException { + writer.flush(); + } + + @Override + public void markCompacted() throws IOException { + writer.finalizeAndClose(); + + idempotentLink(compactingFile, compactedFile); + compactingFile.delete(); + + slog.kv("compactingFile", compactingFile) + .kv("compactedFile", compactedFile) + .info(Events.COMPACTION_MARK_COMPACTED); + } + + @Override + public void scan(EntryLogScanner scanner) throws IOException { + throw new IllegalStateException(exMsg("Scan only valid for recovered log") + .kv("srcLogId", srcLogId) + .kv("dstLogId", dstLogId) + .kv("compactingFile", compactingFile) + .kv("compactedFile", compactedFile) + .kv("completeFile", completeFile).toString()); + } + } + + public static File compactingFile(File directory, int logId) { + return new File(directory, String.format("%x%s", logId, COMPACTING_SUFFIX)); + } + + public static File compactedFile(File directory, int newLogId, int compactedLogId) { + return new File(directory, String.format("%x.log.%x%s", newLogId, + compactedLogId, COMPACTED_SUFFIX)); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java new file mode 100644 index 00000000000..467864d814e --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java @@ -0,0 +1,507 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.RemovalListener; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.stream.Collectors; + +import org.apache.bookkeeper.bookie.AbstractLogCompactor; +import org.apache.bookkeeper.bookie.Bookie.NoEntryException; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.storage.CompactionEntryLog; +import org.apache.bookkeeper.bookie.storage.EntryLogIds; +import org.apache.bookkeeper.bookie.storage.EntryLogIdsImpl; +import org.apache.bookkeeper.bookie.storage.EntryLogScanner; +import org.apache.bookkeeper.bookie.storage.EntryLogger; +import org.apache.bookkeeper.common.util.nativeio.NativeIO; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.StatsLogger; + +/** + * DirectEntryLogger. + */ +public class DirectEntryLogger implements EntryLogger { + private static final String LOGFILE_SUFFIX = ".log"; + private final Slogger slog; + private final File ledgerDir; + private final EntryLogIds ids; + private final ExecutorService writeExecutor; + private final ExecutorService flushExecutor; + private final long maxFileSize; + private final DirectEntryLoggerStats stats; + private final ByteBufAllocator allocator; + private final BufferPool writeBuffers; + private final int readBufferSize; + private final int maxSaneEntrySize; + private final Set unflushedLogs; + + private WriterWithMetadata curWriter; + + private List> pendingFlushes; + private final NativeIO nativeIO; + private final List> allCaches = new CopyOnWriteArrayList<>(); + private final ThreadLocal> caches; + + private static final int NUMBER_OF_WRITE_BUFFERS = 8; + + public DirectEntryLogger(File ledgerDir, + EntryLogIds ids, + NativeIO nativeIO, + ByteBufAllocator allocator, + ExecutorService writeExecutor, + ExecutorService flushExecutor, + long maxFileSize, + int maxSaneEntrySize, + long totalWriteBufferSize, + long totalReadBufferSize, + int readBufferSize, + int numReadThreads, + int maxFdCacheTimeSeconds, + Slogger slogParent, + StatsLogger stats) throws IOException { + this.ledgerDir = ledgerDir; + this.flushExecutor = flushExecutor; + this.writeExecutor = writeExecutor; + this.pendingFlushes = new ArrayList<>(); + this.nativeIO = nativeIO; + this.unflushedLogs = ConcurrentHashMap.newKeySet(); + + this.maxFileSize = maxFileSize; + this.maxSaneEntrySize = maxSaneEntrySize; + this.readBufferSize = Buffer.nextAlignment(readBufferSize); + this.ids = ids; + this.slog = slogParent.kv("directory", ledgerDir).ctx(); + + this.stats = new DirectEntryLoggerStats(stats); + + this.allocator = allocator; + + int singleWriteBufferSize = Buffer.nextAlignment((int) (totalWriteBufferSize / NUMBER_OF_WRITE_BUFFERS)); + this.writeBuffers = new BufferPool(nativeIO, singleWriteBufferSize, NUMBER_OF_WRITE_BUFFERS); + + // The total read buffer memory needs to get split across all the read threads, since the caches + // are thread-specific and we want to ensure we don't pass the total memory limit. + long perThreadBufferSize = totalReadBufferSize / numReadThreads; + + // if the amount of total read buffer size is too low, and/or the number of read threads is too high + // then the perThreadBufferSize can be lower than the readBufferSize causing immediate eviction of readers + // from the cache + if (perThreadBufferSize < readBufferSize) { + slog.kv("reason", "perThreadBufferSize lower than readBufferSize (causes immediate reader cache eviction)") + .kv("totalReadBufferSize", totalReadBufferSize) + .kv("totalNumReadThreads", numReadThreads) + .kv("readBufferSize", readBufferSize) + .kv("perThreadBufferSize", perThreadBufferSize) + .error(Events.ENTRYLOGGER_MISCONFIGURED); + } + + long maxCachedReadersPerThread = perThreadBufferSize / readBufferSize; + long maxCachedReaders = maxCachedReadersPerThread * numReadThreads; + + this.slog + .kv("maxFileSize", maxFileSize) + .kv("maxSaneEntrySize", maxSaneEntrySize) + .kv("totalWriteBufferSize", totalWriteBufferSize) + .kv("singleWriteBufferSize", singleWriteBufferSize) + .kv("totalReadBufferSize", totalReadBufferSize) + .kv("readBufferSize", readBufferSize) + .kv("perThreadBufferSize", perThreadBufferSize) + .kv("maxCachedReadersPerThread", maxCachedReadersPerThread) + .kv("maxCachedReaders", maxCachedReaders) + .info(Events.ENTRYLOGGER_CREATED); + + this.caches = ThreadLocal.withInitial(() -> { + RemovalListener rl = (notification) -> { + try { + notification.getValue().close(); + this.stats.getCloseReaderCounter().inc(); + } catch (IOException ioe) { + slog.kv("logID", notification.getKey()).error(Events.READER_CLOSE_ERROR); + } + }; + Cache cache = CacheBuilder.newBuilder() + .maximumWeight(perThreadBufferSize) + .weigher((key, value) -> readBufferSize) + .removalListener(rl) + .expireAfterAccess(maxFdCacheTimeSeconds, TimeUnit.SECONDS) + .concurrencyLevel(1) // important to avoid too aggressive eviction + .build(); + allCaches.add(cache); + return cache; + }); + } + + @Override + public long addEntry(long ledgerId, ByteBuf buf) throws IOException { + long start = System.nanoTime(); + + long offset; + synchronized (this) { + if (curWriter != null + && curWriter.shouldRoll(buf, maxFileSize)) { + // roll the log. asynchronously flush and close current log + flushAndCloseCurrent(); + curWriter = null; + } + if (curWriter == null) { + int newId = ids.nextId(); + curWriter = new WriterWithMetadata(newDirectWriter(newId), + new EntryLogMetadata(newId), + allocator); + slog.kv("newLogId", newId).info(Events.LOG_ROLL); + } + + offset = curWriter.addEntry(ledgerId, buf); + } + stats.getAddEntryStats().registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + return offset; + } + + @Override + public ByteBuf readEntry(long entryLocation) + throws IOException, NoEntryException { + return internalReadEntry(-1L, -1L, entryLocation, false); + } + + @Override + public ByteBuf readEntry(long ledgerId, long entryId, long entryLocation) + throws IOException, NoEntryException { + return internalReadEntry(ledgerId, entryId, entryLocation, true); + } + + private LogReader getReader(int logId) throws IOException { + Cache cache = caches.get(); + try { + LogReader reader = cache.get(logId, () -> { + this.stats.getOpenReaderCounter().inc(); + return newDirectReader(logId); + }); + + // it is possible though unlikely, that the cache has already cleaned up this cache entry + // during the get operation. This is more likely to happen when there is great demand + // for many separate readers in a low memory environment. + if (reader.isClosed()) { + this.stats.getCachedReadersServedClosedCounter().inc(); + throw new IOException(exMsg("Cached reader already closed").kv("logId", logId).toString()); + } + + return reader; + } catch (ExecutionException ee) { + if (ee.getCause() instanceof IOException) { + throw (IOException) ee.getCause(); + } else { + throw new IOException(exMsg("Error loading reader in cache").kv("logId", logId).toString(), ee); + } + } + } + + private ByteBuf internalReadEntry(long ledgerId, long entryId, long location, boolean validateEntry) + throws IOException, NoEntryException { + int logId = (int) (location >> 32); + int pos = (int) (location & 0xFFFFFFFF); + + long start = System.nanoTime(); + LogReader reader = getReader(logId); + + try { + ByteBuf buf = reader.readEntryAt(pos); + if (validateEntry) { + long thisLedgerId = buf.getLong(0); + long thisEntryId = buf.getLong(8); + if (thisLedgerId != ledgerId + || thisEntryId != entryId) { + throw new IOException( + exMsg("Bad location").kv("location", location) + .kv("expectedLedger", ledgerId).kv("expectedEntry", entryId) + .kv("foundLedger", thisLedgerId).kv("foundEntry", thisEntryId) + .toString()); + } + } + stats.getReadEntryStats().registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + return buf; + } catch (EOFException eof) { + stats.getReadEntryStats().registerFailedEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + throw new NoEntryException( + exMsg("Entry location doesn't exist").kv("location", location).toString(), + ledgerId, entryId); + } + } + + @Override + public void flush() throws IOException { + long start = System.nanoTime(); + Future currentFuture = flushCurrent(); + + List> outstandingFlushes; + synchronized (this) { + outstandingFlushes = this.pendingFlushes; + this.pendingFlushes = new ArrayList<>(); + } + outstandingFlushes.add(currentFuture); + + for (Future f: outstandingFlushes) { + try { + f.get(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new IOException("Interruped while flushing", ie); + } catch (ExecutionException ee) { + if (ee.getCause() instanceof IOException) { + throw (IOException) ee.getCause(); + } else { + throw new IOException("Exception flushing writer", ee); + } + } + } + stats.getFlushStats().registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + } + + private Future flushCurrent() throws IOException { + WriterWithMetadata flushWriter = this.curWriter; + if (flushWriter != null) { + return flushExecutor.submit(() -> { + long start = System.nanoTime(); + try { + flushWriter.flush(); + stats.getWriterFlushStats().registerSuccessfulEvent( + System.nanoTime() - start, TimeUnit.NANOSECONDS); + } catch (Throwable t) { + stats.getWriterFlushStats().registerFailedEvent( + System.nanoTime() - start, TimeUnit.NANOSECONDS); + throw t; + } + return null; + }); + } else { + return CompletableFuture.completedFuture(null); + } + } + + private void flushAndCloseCurrent() throws IOException { + WriterWithMetadata flushWriter; + + CompletableFuture flushPromise = new CompletableFuture<>(); + synchronized (this) { + flushWriter = this.curWriter; + this.curWriter = null; + + pendingFlushes.add(flushPromise); + } + if (flushWriter != null) { + flushExecutor.submit(() -> { + long start = System.nanoTime(); + try { + flushWriter.finalizeAndClose(); + stats.getWriterFlushStats() + .registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + unflushedLogs.remove(flushWriter.logId()); + flushPromise.complete(null); + } catch (Throwable t) { + stats.getWriterFlushStats() + .registerFailedEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + flushPromise.completeExceptionally(t); + } + return null; + }); + } else { + flushPromise.complete(null); + } + } + + @Override + public void close() throws IOException { + flushAndCloseCurrent(); // appends metadata to current log + flush(); // wait for all outstanding flushes + + for (Cache c : allCaches) { + c.invalidateAll(); + } + + writeBuffers.close(); + } + + @Override + public Collection getFlushedLogIds() { + return EntryLogIdsImpl.logIdsInDirectory(ledgerDir).stream() + .filter(logId -> !unflushedLogs.contains(logId)) + .map(i -> Long.valueOf(i)) + .collect(Collectors.toList()); + } + + @Override + public boolean removeEntryLog(long entryLogId) { + checkArgument(entryLogId < Integer.MAX_VALUE, "Entry log id must be an int [%d]", entryLogId); + File file = logFile(ledgerDir, (int) entryLogId); + boolean result = file.delete(); + slog.kv("file", file).kv("logId", entryLogId).kv("result", result).info(Events.LOG_DELETED); + return result; + } + + @Override + public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOException { + checkArgument(entryLogId < Integer.MAX_VALUE, "Entry log id must be an int [%d]", entryLogId); + try (LogReader reader = newDirectReader((int) entryLogId)) { + LogReaderScan.scan(reader, scanner); + } + } + + @Override + public boolean logExists(long logId) { + checkArgument(logId < Integer.MAX_VALUE, "Entry log id must be an int [%d]", logId); + return logFile(ledgerDir, (int) logId).exists(); + } + + @Override + public EntryLogMetadata getEntryLogMetadata(long entryLogId, AbstractLogCompactor.Throttler throttler) + throws IOException { + try { + return readEntryLogIndex(entryLogId); + } catch (IOException e) { + slog.kv("entryLogId", entryLogId).kv("reason", e.getMessage()) + .info(Events.READ_METADATA_FALLBACK); + return scanEntryLogMetadata(entryLogId, throttler); + } + } + + @VisibleForTesting + EntryLogMetadata readEntryLogIndex(long logId) throws IOException { + checkArgument(logId < Integer.MAX_VALUE, "Entry log id must be an int [%d]", logId); + + try (LogReader reader = newDirectReader((int) logId)) { + return LogMetadata.read(reader); + } + } + + @VisibleForTesting + EntryLogMetadata scanEntryLogMetadata(long logId, AbstractLogCompactor.Throttler throttler) throws IOException { + final EntryLogMetadata meta = new EntryLogMetadata(logId); + + // Read through the entry log file and extract the entry log meta + scanEntryLog(logId, new EntryLogScanner() { + @Override + public void process(long ledgerId, long offset, ByteBuf entry) throws IOException { + // add new entry size of a ledger to entry log meta + if (throttler != null) { + throttler.acquire(entry.readableBytes()); + } + meta.addLedgerSize(ledgerId, entry.readableBytes() + Integer.BYTES); + } + + @Override + public boolean accept(long ledgerId) { + return ledgerId >= 0; + } + }); + return meta; + } + + @VisibleForTesting + LogReader newDirectReader(int logId) throws IOException { + return new DirectReader(logId, logFilename(ledgerDir, logId), + allocator, nativeIO, readBufferSize, + maxSaneEntrySize, stats.getReadBlockStats()); + } + + private LogWriter newDirectWriter(int newId) throws IOException { + unflushedLogs.add(newId); + LogWriter writer = new DirectWriter(newId, logFilename(ledgerDir, newId), maxFileSize, + writeExecutor, writeBuffers, nativeIO, slog); + ByteBuf buf = allocator.buffer(Buffer.ALIGNMENT); + try { + Header.writeEmptyHeader(buf); + writer.writeAt(0, buf); + writer.position(buf.capacity()); + } finally { + buf.release(); + } + return writer; + } + + public static File logFile(File directory, int logId) { + return new File(directory, Long.toHexString(logId) + LOGFILE_SUFFIX); + } + + public static String logFilename(File directory, int logId) { + return logFile(directory, logId).toString(); + } + + @Override + public CompactionEntryLog newCompactionLog(long srcLogId) throws IOException { + int dstLogId = ids.nextId(); + return DirectCompactionEntryLog.newLog((int) srcLogId, dstLogId, ledgerDir, + maxFileSize, writeExecutor, writeBuffers, + nativeIO, allocator, slog); + } + + @Override + public Collection incompleteCompactionLogs() { + List logs = new ArrayList<>(); + + for (File f : ledgerDir.listFiles()) { + if (f.getName().endsWith(COMPACTING_SUFFIX)) { + try { + Files.deleteIfExists(f.toPath()); + } catch (IOException ioe) { + slog.kv("file", f).warn(Events.COMPACTION_DELETE_FAILURE); + } + } + + Matcher m = EntryLogIdsImpl.COMPACTED_FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int dstLogId = Integer.parseUnsignedInt(m.group(1), 16); + int srcLogId = Integer.parseUnsignedInt(m.group(2), 16); + + logs.add(DirectCompactionEntryLog.recoverLog(srcLogId, dstLogId, ledgerDir, + readBufferSize, maxSaneEntrySize, + nativeIO, allocator, + stats.getReadBlockStats(), + slog)); + } + } + return logs; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java new file mode 100644 index 00000000000..385766702c0 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java @@ -0,0 +1,174 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE; +import static org.apache.bookkeeper.bookie.BookKeeperServerStats.CATEGORY_SERVER; + +import org.apache.bookkeeper.stats.Counter; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.bookkeeper.stats.annotations.StatsDoc; + +@StatsDoc( + name = BOOKIE_SCOPE, + category = CATEGORY_SERVER, + help = "Direct entry logger stats" +) +class DirectEntryLoggerStats { + private static final String ADD_ENTRY = "entrylog-add-entry"; + private static final String READ_ENTRY = "entrylog-read-entry"; + private static final String FLUSH = "entrylog-flush"; + private static final String WRITER_FLUSH = "entrylog-writer-flush"; + private static final String READ_BLOCK = "entrylog-read-block"; + private static final String READER_OPEN = "entrylog-open-reader"; + private static final String READER_CLOSE = "entrylog-close-reader"; + private static final String CACHED_READER_SERVED_CLOSED = "entrylog-cached-reader-closed"; + + @StatsDoc( + name = ADD_ENTRY, + help = "Operation stats of adding entries to the entry log", + parent = BOOKIE_ADD_ENTRY + ) + private final OpStatsLogger addEntryStats; + + @StatsDoc( + name = READ_ENTRY, + help = "Operation stats of reading entries from the entry log", + parent = BOOKIE_READ_ENTRY + ) + private final ThreadLocal readEntryStats; + + @StatsDoc( + name = FLUSH, + help = "Stats for persisting outstanding entrylog writes to disk" + ) + private final OpStatsLogger flushStats; + + @StatsDoc( + name = WRITER_FLUSH, + help = "Stats for persisting outstanding entrylog writes for a single writer" + ) + private final OpStatsLogger writerFlushStats; + + @StatsDoc( + name = READ_BLOCK, + help = "Stats for reading blocks from disk" + ) + private final ThreadLocal readBlockStats; + + @StatsDoc( + name = READER_OPEN, + help = "Stats for reader open operations" + ) + private final ThreadLocal openReaderStats; + + @StatsDoc( + name = READER_CLOSE, + help = "Stats for reader close operations" + ) + private final ThreadLocal closeReaderStats; + + @StatsDoc( + name = CACHED_READER_SERVED_CLOSED, + help = "Stats for cached readers being served closed" + ) + private final ThreadLocal cachedReadersServedClosed; + + DirectEntryLoggerStats(StatsLogger stats) { + addEntryStats = stats.getOpStatsLogger(ADD_ENTRY); + + flushStats = stats.getOpStatsLogger(FLUSH); + writerFlushStats = stats.getOpStatsLogger(WRITER_FLUSH); + + readEntryStats = new ThreadLocal() { + @Override + public OpStatsLogger initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + .getOpStatsLogger(READ_ENTRY); + } + }; + readBlockStats = new ThreadLocal() { + @Override + public OpStatsLogger initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + .getOpStatsLogger(READ_BLOCK); + } + }; + + openReaderStats = new ThreadLocal() { + @Override + public Counter initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + .getCounter(READER_OPEN); + } + }; + + closeReaderStats = new ThreadLocal() { + @Override + public Counter initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + .getCounter(READER_CLOSE); + } + }; + + cachedReadersServedClosed = new ThreadLocal() { + @Override + public Counter initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + .getCounter(CACHED_READER_SERVED_CLOSED); + } + }; + } + + OpStatsLogger getAddEntryStats() { + return addEntryStats; + } + + OpStatsLogger getFlushStats() { + return flushStats; + } + + OpStatsLogger getWriterFlushStats() { + return writerFlushStats; + } + + OpStatsLogger getReadEntryStats() { + return readEntryStats.get(); + } + + OpStatsLogger getReadBlockStats() { + return readBlockStats.get(); + } + + Counter getOpenReaderCounter() { + return openReaderStats.get(); + } + + Counter getCloseReaderCounter() { + return closeReaderStats.get(); + } + + Counter getCachedReadersServedClosedCounter() { + return cachedReadersServedClosed.get(); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java new file mode 100644 index 00000000000..daab3103c52 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java @@ -0,0 +1,344 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static com.google.common.base.Preconditions.checkState; +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.EOFException; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +import org.apache.bookkeeper.common.util.nativeio.NativeIO; +import org.apache.bookkeeper.common.util.nativeio.NativeIOException; +import org.apache.bookkeeper.stats.OpStatsLogger; + +class DirectReader implements LogReader { + private final ByteBufAllocator allocator; + private final NativeIO nativeIO; + private final Buffer nativeBuffer; + private final String filename; + private final int logId; + private final int fd; + private final int maxSaneEntrySize; + private final OpStatsLogger readBlockStats; + private long currentBlock = -1; + private long currentBlockEnd = -1; + private long maxOffset; + private boolean closed; + + DirectReader(int logId, String filename, ByteBufAllocator allocator, + NativeIO nativeIO, int bufferSize, + int maxSaneEntrySize, OpStatsLogger readBlockStats) throws IOException { + this.nativeIO = nativeIO; + this.allocator = allocator; + this.logId = logId; + this.filename = filename; + this.maxSaneEntrySize = maxSaneEntrySize; + this.readBlockStats = readBlockStats; + + nativeBuffer = new Buffer(nativeIO, bufferSize); + closed = false; + + try { + fd = nativeIO.open(filename, + NativeIO.O_RDONLY | NativeIO.O_DIRECT, + 00755); + checkState(fd >= 0, "Open should throw exception on negative return (%d)", fd); + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("file", filename) + .kv("errno", ne.getErrno()).toString()); + } + refreshMaxOffset(); + } + + @Override + public int logId() { + return logId; + } + + private void clearCache() { + synchronized (nativeBuffer) { + currentBlock = -1; + currentBlockEnd = -1; + } + } + + @Override + public ByteBuf readBufferAt(long offset, int size) throws IOException, EOFException { + ByteBuf buf = allocator.buffer(size); + try { + readIntoBufferAt(buf, offset, size); + } catch (IOException e) { + buf.release(); + throw e; + } + + return buf; + } + + @Override + public void readIntoBufferAt(ByteBuf buffer, long offset, int size) throws IOException, EOFException { + assertValidOffset(offset); + synchronized (nativeBuffer) { + while (size > 0) { + int bytesRead = readBytesIntoBuf(buffer, offset, size); + size -= bytesRead; + offset += bytesRead; + } + } + } + + @Override + public int readIntAt(long offset) throws IOException, EOFException { + assertValidOffset(offset); + synchronized (nativeBuffer) { + if (offset >= currentBlock && offset + Integer.BYTES <= currentBlockEnd) { // fast path + return nativeBuffer.readInt(offsetInBlock(offset)); + } else { // slow path + ByteBuf intBuf = readBufferAt(offset, Integer.BYTES); + try { + return intBuf.getInt(0); + } finally { + intBuf.release(); + } + } + } + } + + @Override + public long readLongAt(long offset) throws IOException, EOFException { + assertValidOffset(offset); + synchronized (nativeBuffer) { + if (offset >= currentBlock && offset + Long.BYTES <= currentBlockEnd) { // fast path + return nativeBuffer.readLong(offsetInBlock(offset)); + } else { // slow path + ByteBuf longBuf = readBufferAt(offset, Long.BYTES); + try { + return longBuf.getLong(0); + } finally { + longBuf.release(); + } + } + } + } + + private int readBytesIntoBuf(ByteBuf buf, long offset, int size) throws IOException, EOFException { + synchronized (nativeBuffer) { + if (offset < currentBlock || offset >= currentBlockEnd) { + readBlock(offset); + } + int offsetInBuffer = offsetInBlock(offset); + int sizeInBuffer = sizeInBlock(offset, size); + if (sizeInBuffer <= 0) { + throw new EOFException(exMsg("Not enough bytes available") + .kv("file", filename) + .kv("fileSize", maxOffset) + .kv("offset", offset) + .kv("size", size).toString()); + } + return nativeBuffer.readByteBuf(buf, offsetInBuffer, size); + } + } + + @Override + public ByteBuf readEntryAt(int offset) throws IOException, EOFException { + assertValidEntryOffset(offset); + int sizeOffset = offset - Integer.BYTES; + if (sizeOffset < 0) { + throw new IOException(exMsg("Invalid offset, buffer size missing") + .kv("file", filename) + .kv("offset", offset).toString()); + } + + int entrySize = readIntAt(sizeOffset); + if (entrySize == 0) { + // reading an entry with size 0 may mean reading from preallocated + // space. if we receive an offset in preallocated space, it may + // mean that a write has occurred and been flushed, but our view + // of that block is out of date. So clear the cache and let it be + // loaded again. + clearCache(); + entrySize = readIntAt(sizeOffset); + } + if (entrySize > maxSaneEntrySize || entrySize <= 0) { + throw new IOException(exMsg("Invalid entry size") + .kv("file", filename) + .kv("offset", offset) + .kv("maxSaneEntrySize", maxSaneEntrySize) + .kv("readEntrySize", entrySize).toString()); + } + return readBufferAt(offset, entrySize); + } + + void readBlock(long offset) throws IOException { + final int blockSize = nativeBuffer.size(); + assertValidBlockSize(blockSize); + final long blockStart = offset & ~(blockSize - 1); + + if (blockStart + blockSize > maxOffset) { + // Check if there's new data in the file + refreshMaxOffset(); + } + final long bytesAvailable = maxOffset > blockStart ? maxOffset - blockStart : 0; + final long startNs = System.nanoTime(); + + long bufferOffset = 0; + long bytesToRead = Math.min(blockSize, bytesAvailable); + long bytesOutstanding = bytesToRead; + int attempts = 0; + long bytesRead = -1; + try { + while (true) { + attempts++; + + long readSize = blockSize - bufferOffset; + long pointerWithOffset = nativeBuffer.pointer(bufferOffset, readSize); + bytesRead = nativeIO.pread(fd, pointerWithOffset, + readSize, + blockStart + bufferOffset); + // offsets and counts must be aligned, so ensure that if we + // get a short read, we don't throw off the alignment. For example + // if we're trying to read 12K and we only managed 100 bytes, + // we don't progress the offset or outstanding at all. However, if we + // read 4196 bytes, we can progress the offset by 4KB and the outstanding + // bytes will then be 100. + // the only non-short read that isn't aligned is the bytes at the end of + // of the file, which is why we don't align before we check if we should + // exit the loop + if ((bytesOutstanding - bytesRead) <= 0) { + break; + } + bytesOutstanding -= bytesRead & Buffer.ALIGNMENT; + bufferOffset += bytesRead & Buffer.ALIGNMENT; + } + } catch (NativeIOException ne) { + readBlockStats.registerFailedEvent(System.nanoTime() - startNs, TimeUnit.NANOSECONDS); + throw new IOException(exMsg(ne.getMessage()) + .kv("requestedBytes", blockSize) + .kv("offset", blockStart) + .kv("expectedBytes", Math.min(blockSize, bytesAvailable)) + .kv("bytesOutstanding", bytesOutstanding) + .kv("bufferOffset", bufferOffset) + .kv("file", filename) + .kv("fd", fd) + .kv("errno", ne.getErrno()).toString()); + } + readBlockStats.registerSuccessfulEvent(System.nanoTime() - startNs, TimeUnit.NANOSECONDS); + currentBlock = blockStart; + currentBlockEnd = blockStart + Math.min(blockSize, bytesAvailable); + } + + @Override + public void close() throws IOException { + synchronized (nativeBuffer) { + nativeBuffer.free(); + } + + try { + int ret = nativeIO.close(fd); + checkState(ret == 0, "Close should throw exception on non-zero return (%d)", ret); + closed = true; + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("file", filename) + .kv("errno", ne.getErrno()).toString()); + } + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public long maxOffset() { + return maxOffset; + } + + private void refreshMaxOffset() throws IOException { + try { + long ret = nativeIO.lseek(fd, 0, NativeIO.SEEK_END); + checkState(ret >= 0, + "Lseek should throw exception on negative return (%d)", ret); + synchronized (this) { + maxOffset = ret; + } + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("file", filename) + .kv("fd", fd) + .kv("errno", ne.getErrno()).toString()); + } + } + + private int offsetInBlock(long offset) { + long blockOffset = offset - currentBlock; + if (blockOffset < 0 || blockOffset > Integer.MAX_VALUE) { + throw new IllegalArgumentException(exMsg("Invalid offset passed") + .kv("offset", offset).kv("blockOffset", blockOffset) + .kv("currentBlock", currentBlock).toString()); + } + return (int) blockOffset; + } + + private int sizeInBlock(long offset, int size) { + if (offset > currentBlockEnd || offset < currentBlock) { + throw new IllegalArgumentException(exMsg("Invalid offset passed") + .kv("offset", offset) + .kv("currentBlock", currentBlock) + .kv("currentBlockEnd", currentBlockEnd).toString()); + } + + long blockOffset = offset - currentBlock; + long available = currentBlockEnd - offset; + checkState(available <= Integer.MAX_VALUE, "Available(%d) must be less than max int", available); + return Math.min(size, (int) available); + } + + private static void assertValidOffset(long offset) { + if (offset < 0) { + throw new IllegalArgumentException( + exMsg("Offset can't be negative").kv("offset", offset).toString()); + } + } + + private static void assertValidEntryOffset(long offset) { + assertValidOffset(offset); + if (offset > Integer.MAX_VALUE) { + throw new IllegalArgumentException( + exMsg("Entry offset must be less than max int").kv("offset", offset).toString()); + } + } + + private static void assertValidBlockSize(int blockSize) { + boolean valid = blockSize > 0 && Buffer.isAligned(blockSize); + if (!valid) { + throw new IllegalArgumentException( + exMsg("Invalid block size, must be power of 2") + .kv("blockSize", blockSize) + .kv("minBlockSize", Buffer.ALIGNMENT).toString()); + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java new file mode 100644 index 00000000000..37a54d087e3 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -0,0 +1,322 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import io.netty.buffer.ByteBuf; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; + +import org.apache.bookkeeper.common.util.nativeio.NativeIO; +import org.apache.bookkeeper.common.util.nativeio.NativeIOException; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.commons.lang3.SystemUtils; + +class DirectWriter implements LogWriter { + final NativeIO nativeIO; + final int fd; + final int id; + final String filename; + final BufferPool bufferPool; + final ExecutorService writeExecutor; + final Object bufferLock = new Object(); + final List> outstandingWrites = new ArrayList>(); + Buffer nativeBuffer; + long offset; + private static volatile boolean useFallocate = true; + + DirectWriter(int id, + String filename, + long maxFileSize, + ExecutorService writeExecutor, + BufferPool bufferPool, + NativeIO nativeIO, Slogger slog) throws IOException { + checkArgument(maxFileSize > 0, "Max file size (%d) must be positive"); + this.id = id; + this.filename = filename; + this.writeExecutor = writeExecutor; + this.nativeIO = nativeIO; + + offset = 0; + + try { + fd = nativeIO.open(filename, + NativeIO.O_CREAT | NativeIO.O_WRONLY | NativeIO.O_DIRECT, + 00755); + checkState(fd >= 0, "Open should have thrown exception, fd is invalid : %d", fd); + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()).kv("file", filename) + .kv("errno", ne.getErrno()).toString(), ne); + } + + if (useFallocate) { + if (!SystemUtils.IS_OS_LINUX) { + useFallocate = false; + slog.warn(Events.FALLOCATE_NOT_AVAILABLE); + } else { + try { + int ret = nativeIO.fallocate(fd, NativeIO.FALLOC_FL_ZERO_RANGE, 0, maxFileSize); + checkState(ret == 0, "Exception should have been thrown on non-zero ret: %d", ret); + } catch (NativeIOException ex) { + // fallocate(2) is not supported on all filesystems. Since this is an optimization, disable + // subsequent usage instead of failing the operation. + useFallocate = false; + slog.kv("message", ex.getMessage()) + .kv("file", filename) + .kv("errno", ex.getErrno()) + .warn(Events.FALLOCATE_NOT_AVAILABLE); + } + } + } + + this.bufferPool = bufferPool; + this.nativeBuffer = bufferPool.acquire(); + } + + @Override + public int logId() { + return id; + } + + @Override + public void writeAt(long offset, ByteBuf buf) throws IOException { + checkArgument(Buffer.isAligned(offset), + "Offset to writeAt must be aligned to %d: %d is not", Buffer.ALIGNMENT, offset); + checkArgument(Buffer.isAligned(buf.readableBytes()), + "Buffer must write multiple of alignment bytes (%d), %d is not", + Buffer.ALIGNMENT, buf.readableBytes()); + Buffer tmpBuffer = bufferPool.acquire(); + int bytesToWrite = buf.readableBytes(); + tmpBuffer.reset(); + tmpBuffer.writeByteBuf(buf); + Future f = writeExecutor.submit(() -> { + try { + int ret = nativeIO.pwrite(fd, tmpBuffer.pointer(), bytesToWrite, offset); + if (ret != bytesToWrite) { + throw new IOException(exMsg("Incomplete write") + .kv("filename", filename) + .kv("writeSize", bytesToWrite) + .kv("bytesWritten", ret) + .kv("offset", offset).toString()); + } + } catch (NativeIOException ne) { + throw new IOException(exMsg("Write error") + .kv("filename", filename) + .kv("writeSize", bytesToWrite) + .kv("errno", ne.getErrno()) + .kv("offset", offset).toString()); + } finally { + bufferPool.release(tmpBuffer); + } + return null; + }); + addOutstandingWrite(f); + } + + @Override + public int writeDelimited(ByteBuf buf) throws IOException { + synchronized (bufferLock) { + if (!nativeBuffer.hasSpace(serializedSize(buf))) { + flushBuffer(); + } + + int readable = buf.readableBytes(); + long bufferPosition = position() + Integer.BYTES; + if (bufferPosition > Integer.MAX_VALUE) { + throw new IOException(exMsg("Cannot write past max int") + .kv("filename", filename) + .kv("writeSize", readable) + .kv("position", bufferPosition) + .toString()); + } + nativeBuffer.writeInt(readable); + nativeBuffer.writeByteBuf(buf); + return (int) bufferPosition; + } + } + + @Override + public void position(long offset) throws IOException { + synchronized (bufferLock) { + if (nativeBuffer != null && nativeBuffer.position() > 0) { + flushBuffer(); + } + if ((offset % Buffer.ALIGNMENT) != 0) { + throw new IOException(exMsg("offset must be multiple of alignment") + .kv("offset", offset) + .kv("alignment", Buffer.ALIGNMENT) + .toString()); + } + this.offset = offset; + } + } + + @Override + public long position() { + synchronized (bufferLock) { + return this.offset + (nativeBuffer != null ? nativeBuffer.position() : 0); + } + } + + @Override + public void flush() throws IOException { + flushBuffer(); + + waitForOutstandingWrites(); + + try { + int ret = nativeIO.fsync(fd); + checkState(ret == 0, "Fsync should throw exception on non-zero return (%d)", ret); + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("file", filename) + .kv("errno", ne.getErrno()).toString()); + } + } + + @Override + public void close() throws IOException { + synchronized (bufferLock) { + if (nativeBuffer != null && nativeBuffer.position() > 0) { + flushBuffer(); + } + } + + try { + int ret = nativeIO.close(fd); + checkState(ret == 0, "Close should throw exception on non-zero return (%d)", ret); + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("file", filename) + .kv("errno", ne.getErrno()).toString()); + } + synchronized (bufferLock) { + bufferPool.release(nativeBuffer); + nativeBuffer = null; + } + } + + private void addOutstandingWrite(Future toAdd) throws IOException { + synchronized (outstandingWrites) { + outstandingWrites.add(toAdd); + + Iterator> iter = outstandingWrites.iterator(); + while (iter.hasNext()) { // clear out completed futures + Future f = iter.next(); + if (f.isDone()) { + waitForFuture(f); + iter.remove(); + } else { + break; + } + } + } + } + + private void waitForOutstandingWrites() throws IOException { + synchronized (outstandingWrites) { + Iterator> iter = outstandingWrites.iterator(); + while (iter.hasNext()) { // clear out completed futures + Future f = iter.next(); + waitForFuture(f); + iter.remove(); + } + } + } + + private void waitForFuture(Future f) throws IOException { + try { + f.get(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new IOException(ie); + } catch (Throwable t) { + if (t.getCause() instanceof IOException) { + throw (IOException) t.getCause(); + } else { + throw new IOException(t); + } + } + } + + private void flushBuffer() throws IOException { + synchronized (bufferLock) { + if (this.nativeBuffer != null) { + int bytesToWrite = this.nativeBuffer.padToAlignment(); + if (bytesToWrite == 0) { + return; + } + + Buffer bufferToFlush = this.nativeBuffer; + this.nativeBuffer = null; + + long offsetToWrite = offset; + offset += bytesToWrite; + + Future f = writeExecutor.submit(() -> { + try { + if (bytesToWrite <= 0) { + return null; + } + int ret = nativeIO.pwrite(fd, bufferToFlush.pointer(), bytesToWrite, offsetToWrite); + if (ret != bytesToWrite) { + throw new IOException(exMsg("Incomplete write") + .kv("filename", filename) + .kv("pointer", bufferToFlush.pointer()) + .kv("offset", offsetToWrite) + .kv("writeSize", bytesToWrite) + .kv("bytesWritten", ret).toString()); + } + } catch (NativeIOException ne) { + throw new IOException(exMsg(ne.getMessage()) + .kv("filename", filename) + .kv("offset", offsetToWrite) + .kv("writeSize", bytesToWrite) + .kv("pointer", bufferToFlush.pointer()) + .kv("errno", ne.getErrno()).toString(), ne); + } finally { + bufferPool.release(bufferToFlush); + } + return null; + }); + addOutstandingWrite(f); + + // must acquire after triggering the write + // otherwise it could try to acquire a buffer without kicking off + // a subroutine that will free another + this.nativeBuffer = bufferPool.acquire(); + } + } + } + + @Override + public int serializedSize(ByteBuf buf) { + return buf.readableBytes() + Integer.BYTES; + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java new file mode 100644 index 00000000000..5fdc96ff3ce --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java @@ -0,0 +1,145 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +/** + * Events. + */ +public enum Events { + /** + * Fallocate is not available on this host. This generally indicates that the process is running on a + * non-Linux operating system. The lack of fallocate means that the filesystem will have to do more + * bookkeeping as data is written to the file, which will slow down writes. + */ + FALLOCATE_NOT_AVAILABLE, + + /** + * EntryLog ID candidates selected. These are the set entry log ID that subsequent entry log files + * will use. To find the candidates, the bookie lists all the log ids which have already been used, + * and finds the longest contiguous block of free ids. Over the lifetime of a bookie, a log id can + * be reused. This is not a problem, as the ids are only referenced from the index, and an + * entry log file will not be deleted if there are still references to it in the index. + * Generally candidates are selected at bookie boot, but they may also be selected at a later time + * if the current set of candidates is depleted. + */ + ENTRYLOG_IDS_CANDIDATES_SELECTED, + + /** + * The entrylogger({@link org.apache.bookkeeper.bookie.storage.EntryLogger}) has been created. + * This occurs during bookie bootup, and the same entry logger will be used for the duration of + * the bookie process's lifetime. + */ + ENTRYLOGGER_CREATED, + + /** + * The entrylogger has been configured in a way that will likely result in errors during operation. + */ + ENTRYLOGGER_MISCONFIGURED, + + /** + * The entrylogger has started writing the a new log file. The previous log file may not + * be entirely flushed when this is called, though they will be after an explicit flush call. + */ + LOG_ROLL, + + /** + * A log file has been deleted. This happens as a result of GC, when all entries in the file + * belong to deleted ledgers, or compaction, where the live entries have been copied to a new + * log. + */ + LOG_DELETED, + + /** + * An error occurred closing an entrylog reader. This is non-fatal but it may leak the file handle + * and the memory buffer of the reader in question. + */ + READER_CLOSE_ERROR, + + /** + * An attempt to read entrylog metadata failed. Falling back to scanning the log to get the metadata. + * This can occur if a bookie crashes before closing the entrylog cleanly. + */ + READ_METADATA_FALLBACK, + + /** + * A new entrylog has been created. The filename has the format [dstLogId].compacting, where dstLogId is + * a new unique log ID. Entrylog compaction will copy live entries from an existing src log to this new + * compacting destination log. There is a 1-1 relationship between source logs and destination log logs. + * Once the copy completes, the compacting log will be marked as compacted by renaming the file to + * [dstLogId].log.[srcLogId].compacted, where srcLogId is the ID of the entrylog from which the live entries + * were copied. A new hardlink, [dstLogId].log, is created to point to the same inode, making the entry + * log available to be read. The compaction algorithm then updates the index with the offsets of the entries + * in the compacted destination log. Once complete, the index is flushed and all intermediate files (links) + * are deleted along with the original source log file. + * The entry copying phase of compaction is expensive. The renaming and linking in the algorithm exists so + * if a failure occurs after copying has completed, the work will not need to be redone. + */ + COMPACTION_LOG_CREATED, + + /** + * A partially compacted log has been recovered. The log file is of the format [dstLogId].log.[srcLogId].compacted. + * The log will be scanned and the index updated with the offsets of the entries in the log. Once complete, the + * log with ID srcLogId is deleted. + *

+ * See {@link #COMPACTION_LOG_CREATED} for more information on compaction. + */ + COMPACTION_LOG_RECOVERED, + + /** + * A compaction log has been marked as compacted. A log is marked as compacted by renaming from [dstLogId].log to + * [dstLogId].log.[srcLogId].compacted. All live entries from the src log have been successfully copied to the + * destination log, at this point. + *

+ * See {@link #COMPACTION_LOG_CREATED} for more information on compaction. + */ + COMPACTION_MARK_COMPACTED, + + /** + * A compacted log has been made available for reads. A log is made available by creating a hardlink + * pointing from [dstLogId].log, to [dstLogId].log.[srcLogId].compacted. These files, pointing to the + * same inode, will continue to exist until the compaction operation is complete. + *

+ * A reader with a valid offset will now be able to read from this log, so the index can be updated. + *

+ * See {@link #COMPACTION_LOG_CREATED} for more information on compaction. + */ + COMPACTION_MAKE_AVAILABLE, + + /** + * Compaction has been completed for a log. + * All intermediatory files are deleted, along with the src entrylog file. + *

+ * See {@link #COMPACTION_LOG_CREATED} for more information on compaction. + */ + COMPACTION_COMPLETE, + + /** + * Failed to delete files while aborting a compaction operation. While this is not fatal, it + * can mean that there are issues writing to the filesystem that need to be investigated. + */ + COMPACTION_ABORT_EXCEPTION, + + /** + * Failed to delete files while completing a compaction operation. While this is not fatal, it + * can mean that there are issues writing to the filesystem that need to be investigated. + */ + COMPACTION_DELETE_FAILURE, +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Header.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Header.java new file mode 100644 index 00000000000..e15a301826a --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Header.java @@ -0,0 +1,105 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import java.io.IOException; + +/** + * The 1K block at the head of the entry logger file + * that contains the fingerprint and meta-data. + * + *

+ * Header is composed of:
+ * Fingerprint: 4 bytes "BKLO"
+ * Log file HeaderVersion enum: 4 bytes
+ * Ledger map offset: 8 bytes
+ * Ledgers Count: 4 bytes
+ * 
+ */ +class Header { + static final int LOGFILE_LEGACY_HEADER_SIZE = 1024; + static final int LOGFILE_DIRECT_HEADER_SIZE = Buffer.ALIGNMENT; + static final int HEADER_VERSION_OFFSET = 4; + static final int LEDGERS_MAP_OFFSET = HEADER_VERSION_OFFSET + Integer.BYTES; + static final int LEDGER_COUNT_OFFSET = LEDGERS_MAP_OFFSET + Long.BYTES; + static final int HEADER_V0 = 0; // Old log file format (no ledgers map index) + static final int HEADER_V1 = 1; // Introduced ledger map index + static final int HEADER_CURRENT_VERSION = HEADER_V1; + + static final byte[] EMPTY_HEADER = new byte[LOGFILE_DIRECT_HEADER_SIZE]; + static { + ByteBuf buf = Unpooled.wrappedBuffer(EMPTY_HEADER); + buf.setByte(0, 'B'); + buf.setByte(1, 'K'); + buf.setByte(2, 'L'); + buf.setByte(3, 'O'); + buf.setInt(HEADER_VERSION_OFFSET, HEADER_V1); + // legacy header size is 1024, while direct is 4096 so that it can be written as a single block + // to avoid legacy failing when it encounters the header in direct, create a dummy entry, which + // skips to the start of the second block + buf.setInt(LOGFILE_LEGACY_HEADER_SIZE, (buf.capacity() - LOGFILE_LEGACY_HEADER_SIZE) - Integer.BYTES); + buf.setLong(LOGFILE_LEGACY_HEADER_SIZE + Integer.BYTES, LogMetadata.INVALID_LID); + }; + static int extractVersion(ByteBuf header) throws IOException { + assertFingerPrint(header); + return header.getInt(HEADER_VERSION_OFFSET); + } + + static long extractLedgerMapOffset(ByteBuf header) throws IOException { + assertFingerPrint(header); + return header.getLong(LEDGERS_MAP_OFFSET); + } + + static int extractLedgerCount(ByteBuf header) throws IOException { + assertFingerPrint(header); + return header.getInt(LEDGER_COUNT_OFFSET); + } + + static void assertFingerPrint(ByteBuf header) throws IOException { + if (header.getByte(0) != 'B' + || header.getByte(1) != 'K' + || header.getByte(2) != 'L' + || header.getByte(3) != 'O') { + throw new IOException(exMsg("Bad fingerprint (should be BKLO)") + .kv("byte0", header.getByte(0)) + .kv("byte1", header.getByte(0)) + .kv("byte2", header.getByte(0)) + .kv("byte3", header.getByte(0)) + .toString()); + } + } + + static void writeEmptyHeader(ByteBuf header) throws IOException { + header.writeBytes(EMPTY_HEADER); + } + + static void writeHeader(ByteBuf header, + long ledgerMapOffset, int ledgerCount) throws IOException { + header.writeBytes(EMPTY_HEADER); + header.setLong(LEDGERS_MAP_OFFSET, ledgerMapOffset); + header.setInt(LEDGER_COUNT_OFFSET, ledgerCount); + } + +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java new file mode 100644 index 00000000000..e78e67f0cf5 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java @@ -0,0 +1,192 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.IOException; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap; +import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap.BiConsumerLong; + +class LogMetadata { + /** + * Ledgers map is composed of multiple parts that can be split into separated entries. Each of them is composed of: + * + *
+     * length: (4 bytes) [0-3]
+     * ledger id (-1): (8 bytes) [4 - 11]
+     * entry id: (8 bytes) [12-19]
+     * num ledgers stored in current metadata entry: (4 bytes) [20 - 23]
+     * ledger entries: sequence of (ledgerid, size) (8 + 8 bytes each) [24..]
+     * 
+ */ + static final int LEDGERS_MAP_HEADER_SIZE = 4 + 8 + 8 + 4; + static final int LEDGERS_MAP_ENTRY_SIZE = 8 + 8; + + // Break the ledgers map into multiple batches, each of which can contain up to 10K ledgers + static final int LEDGERS_MAP_MAX_BATCH_SIZE = 10000; + static final int LEDGERS_MAP_MAX_MAP_SIZE = + LEDGERS_MAP_HEADER_SIZE + LEDGERS_MAP_ENTRY_SIZE * LEDGERS_MAP_MAX_BATCH_SIZE; + + static final long INVALID_LID = -1L; + // EntryId used to mark an entry (belonging to INVALID_ID) + // as a component of the serialized ledgers map + static final long LEDGERS_MAP_ENTRY_ID = -2L; + + static void write(LogWriter writer, + EntryLogMetadata metadata, + ByteBufAllocator allocator) + throws IOException { + long ledgerMapOffset = writer.position(); + ConcurrentLongLongHashMap ledgersMap = metadata.getLedgersMap(); + int numberOfLedgers = (int) ledgersMap.size(); + + // Write the ledgers map into several batches + final ByteBuf serializedMap = allocator.buffer(LEDGERS_MAP_MAX_BATCH_SIZE); + BiConsumerLong writingConsumer = new BiConsumerLong() { + int remainingLedgers = numberOfLedgers; + boolean startNewBatch = true; + int remainingInBatch = 0; + + @Override + public void accept(long ledgerId, long size) { + if (startNewBatch) { + int batchSize = Math.min(remainingLedgers, LEDGERS_MAP_MAX_BATCH_SIZE); + serializedMap.clear(); + serializedMap.writeLong(INVALID_LID); + serializedMap.writeLong(LEDGERS_MAP_ENTRY_ID); + serializedMap.writeInt(batchSize); + + startNewBatch = false; + remainingInBatch = batchSize; + } + // Dump the ledger in the current batch + serializedMap.writeLong(ledgerId); + serializedMap.writeLong(size); + --remainingLedgers; + + if (--remainingInBatch == 0) { + // Close current batch + try { + writer.writeDelimited(serializedMap); + } catch (IOException e) { + throw new RuntimeException(e); + } + startNewBatch = true; + } + } + }; + try { + ledgersMap.forEach(writingConsumer); + } catch (RuntimeException e) { + if (e.getCause() instanceof IOException) { + throw (IOException) e.getCause(); + } else { + throw e; + } + } finally { + serializedMap.release(); + } + writer.flush(); + + ByteBuf buf = allocator.buffer(Buffer.ALIGNMENT); + try { + Header.writeHeader(buf, ledgerMapOffset, numberOfLedgers); + writer.writeAt(0, buf); + } finally { + buf.release(); + } + } + + static EntryLogMetadata read(LogReader reader) throws IOException { + ByteBuf header = reader.readBufferAt(0, Header.LOGFILE_LEGACY_HEADER_SIZE); + try { + int headerVersion = Header.extractVersion(header); + if (headerVersion < Header.HEADER_V1) { + throw new IOException(exMsg("Old log file header").kv("headerVersion", headerVersion).toString()); + } + long ledgerMapOffset = Header.extractLedgerMapOffset(header); + if (ledgerMapOffset > Integer.MAX_VALUE) { + throw new IOException(exMsg("ledgerMapOffset too high").kv("ledgerMapOffset", ledgerMapOffset) + .kv("maxOffset", Integer.MAX_VALUE).toString()); + } + if (ledgerMapOffset <= 0) { + throw new IOException(exMsg("ledgerMap never written").kv("ledgerMapOffset", ledgerMapOffset) + .toString()); + } + int ledgerCount = Header.extractLedgerCount(header); + + long offset = ledgerMapOffset; + EntryLogMetadata meta = new EntryLogMetadata(reader.logId()); + while (offset < reader.maxOffset()) { + int mapSize = reader.readIntAt((int) offset); + if (mapSize >= LogMetadata.LEDGERS_MAP_MAX_MAP_SIZE) { + throw new IOException(exMsg("ledgerMap too large") + .kv("maxSize", LogMetadata.LEDGERS_MAP_MAX_MAP_SIZE) + .kv("mapSize", mapSize).toString()); + } else if (mapSize <= 0) { + break; + } + offset += Integer.BYTES; + + ByteBuf ledgerMapBuffer = reader.readBufferAt(offset, mapSize); + try { + offset += mapSize; + + long ledgerId = ledgerMapBuffer.readLong(); + if (ledgerId != LogMetadata.INVALID_LID) { + throw new IOException(exMsg("Bad ledgerID").kv("ledgerId", ledgerId).toString()); + } + long entryId = ledgerMapBuffer.readLong(); + if (entryId != LogMetadata.LEDGERS_MAP_ENTRY_ID) { + throw new IOException(exMsg("Unexpected entry ID. Expected special value") + .kv("entryIdRead", entryId) + .kv("entryIdExpected", LogMetadata.LEDGERS_MAP_ENTRY_ID).toString()); + } + int countInBatch = ledgerMapBuffer.readInt(); + for (int i = 0; i < countInBatch; i++) { + ledgerId = ledgerMapBuffer.readLong(); + long size = ledgerMapBuffer.readLong(); + meta.addLedgerSize(ledgerId, size); + } + if (ledgerMapBuffer.isReadable()) { + throw new IOException(exMsg("ledgerMapSize didn't match content") + .kv("expectedCount", countInBatch) + .kv("bufferSize", mapSize) + .kv("bytesRemaining", ledgerMapBuffer.readableBytes()) + .toString()); + } + } finally { + ledgerMapBuffer.release(); + } + } + return meta; + } catch (IOException ioe) { + throw new IOException(exMsg("Error reading index").kv("logId", reader.logId()) + .kv("reason", ioe.getMessage()).toString(), ioe); + } finally { + header.release(); + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReader.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReader.java new file mode 100644 index 00000000000..98bb23aa745 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReader.java @@ -0,0 +1,80 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import io.netty.buffer.ByteBuf; +import java.io.EOFException; +import java.io.IOException; + +/** + * Interface for reading from a bookkeeper entry log. + */ +public interface LogReader extends AutoCloseable { + /** + * @return the id of the log being read from. + */ + int logId(); + + /** + * @return the maximum offset in the file that can be read from. + */ + long maxOffset(); + + /** + * Read a buffer from the file. It is the responsibility of the caller to release + * the returned buffer. + * @param offset the offset to read at + * @param size the number of bytes to read + * @return a bytebuf. The caller must release. + */ + ByteBuf readBufferAt(long offset, int size) throws IOException, EOFException; + + void readIntoBufferAt(ByteBuf buffer, long offset, int size) throws IOException, EOFException; + + /** + * Read an integer at a given offset. + * @param offset the offset to read from. + * @return the integer at that offset. + */ + int readIntAt(long offset) throws IOException, EOFException; + + /** + * Read an long at a given offset. + * @param offset the offset to read from. + * @return the long at that offset. + */ + long readLongAt(long offset) throws IOException, EOFException; + + /** + * Read an entry at a given offset. + * The size of the entry must be at (offset - Integer.BYTES). + * The payload of the entry starts at offset. + * It is the responsibility of the caller to release the returned buffer. + * @param offset the offset at which to read the entry. + * @return a bytebuf. The caller must release. + */ + ByteBuf readEntryAt(int offset) throws IOException, EOFException; + + @Override + void close() throws IOException; + + boolean isClosed(); +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReaderScan.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReaderScan.java new file mode 100644 index 00000000000..eadc02f3da1 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogReaderScan.java @@ -0,0 +1,61 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.PooledByteBufAllocator; +import java.io.IOException; +import org.apache.bookkeeper.bookie.storage.EntryLogScanner; + +class LogReaderScan { + static void scan(LogReader reader, EntryLogScanner scanner) throws IOException { + int offset = Header.LOGFILE_LEGACY_HEADER_SIZE; + + ByteBuf entry = PooledByteBufAllocator.DEFAULT.directBuffer(16 * 1024 * 1024); + + try { + while (offset < reader.maxOffset()) { + int initOffset = offset; + int entrySize = reader.readIntAt(offset); + if (entrySize < 0) { // padding, skip it + offset = Buffer.nextAlignment(offset); + continue; + } else if (entrySize == 0) { // preallocated space, we're done + break; + } + + // The 4 bytes for the entrySize need to be added only after we + // have realigned on the block boundary. + offset += Integer.BYTES; + + entry.clear(); + reader.readIntoBufferAt(entry, offset, entrySize); + long ledgerId = entry.getLong(0); + if (ledgerId >= 0 && scanner.accept(ledgerId)) { + scanner.process(ledgerId, initOffset, entry); + } + offset += entrySize; + } + } finally { + entry.release(); + } + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java new file mode 100644 index 00000000000..6135b146ef3 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java @@ -0,0 +1,81 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import io.netty.buffer.ByteBuf; +import java.io.IOException; + +/** + * Interface for writing data to a bookkeeper entry log. + */ +interface LogWriter extends AutoCloseable { + /** + * Return the ID of the log being written. + */ + int logId(); + + /** + * Write the contents of a buffer at a predefined position in the log. + * Both the position and the size of the buffer must be page aligned (i.e. to 4096). + */ + void writeAt(long offset, ByteBuf buf) throws IOException; + + /** + * Write a delimited buffer the the log. The size of the buffer is first + * written and then the buffer itself. + * Note that the returned offset is for the buffer itself, not the size. + * So, if a buffer is written at the start of the file, the returned offset + * will be 4, not 0. + * The returned offset is an int. Consequently, entries can only be written + * in the first Integer.MAX_VALUE bytes of the file. This is due to how + * offsets are stored in the index. + * + * @return the offset of the buffer within the file. + */ + int writeDelimited(ByteBuf buf) throws IOException; + + /** + * @return the number of bytes consumed by the buffer when written with #writeDelimited + */ + int serializedSize(ByteBuf buf); + + /** + * The current offset within the log at which the next call to #writeDelimited will + * start writing. + */ + long position() throws IOException; + + /** + * Set the offset within the log at which the next call to #writeDelimited will start writing. + */ + void position(long offset) throws IOException; + + /** + * Flush all buffered writes to disk. This call must ensure that the bytes are actually on + * disk before returning. + */ + void flush() throws IOException; + + /** + * Close any held resources. + */ + void close() throws IOException; +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/WriterWithMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/WriterWithMetadata.java new file mode 100644 index 00000000000..5f4d1b98433 --- /dev/null +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/WriterWithMetadata.java @@ -0,0 +1,79 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static com.google.common.base.Preconditions.checkState; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.IOException; +import org.apache.bookkeeper.bookie.EntryLogMetadata; + +/** + * WriterWithMetadata. + */ +class WriterWithMetadata { + private final LogWriter writer; + private final EntryLogMetadata metadata; + private final ByteBufAllocator allocator; + + WriterWithMetadata(LogWriter writer, EntryLogMetadata metadata, + ByteBufAllocator allocator) throws IOException { + this.writer = writer; + this.metadata = metadata; + this.allocator = allocator; + + ByteBuf buf = allocator.buffer(Buffer.ALIGNMENT); + try { + Header.writeEmptyHeader(buf); + writer.writeAt(0, buf); + writer.position(buf.capacity()); + } finally { + buf.release(); + } + } + + int logId() { + return writer.logId(); + } + + boolean shouldRoll(ByteBuf entry, long rollThreshold) throws IOException { + return (writer.position() + writer.serializedSize(entry)) > rollThreshold; + } + + long addEntry(long ledgerId, ByteBuf entry) throws IOException { + int size = entry.readableBytes(); + metadata.addLedgerSize(ledgerId, size + Integer.BYTES); + long offset = writer.writeDelimited(entry); + checkState(offset < Integer.MAX_VALUE, "Offsets can't be higher than max int (%d)", offset); + return ((long) writer.logId()) << 32 | offset; + } + + void flush() throws IOException { + writer.flush(); + } + + void finalizeAndClose() throws IOException { + writer.flush(); + LogMetadata.write(writer, metadata, allocator); + writer.close(); + } +} diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 7d33e5eb5e9..425f0ef2c75 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -39,6 +39,7 @@ import java.util.EnumSet; import java.util.List; import java.util.PrimitiveIterator.OfLong; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -49,18 +50,24 @@ import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.Checkpointer; +import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.GarbageCollectionStatus; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.bookie.LedgerCache; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.StateManager; +import org.apache.bookkeeper.bookie.storage.EntryLogIdsImpl; +import org.apache.bookkeeper.bookie.storage.EntryLogger; +import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType; import org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.common.util.Watcher; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; +import org.apache.bookkeeper.slogger.slf4j.Slf4jSlogger; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -68,7 +75,6 @@ import org.apache.bookkeeper.util.DiskChecker; import org.apache.commons.lang3.StringUtils; - /** * Implementation of LedgerStorage that uses RocksDB to keep the indexes for entries stored in EntryLogs. */ @@ -78,6 +84,18 @@ public class DbLedgerStorage implements LedgerStorage { public static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb"; public static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb"; + static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize"; + public static final String DIRECT_IO_ENTRYLOGGER = "dbStorage_directIOEntryLogger"; + public static final String DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB = + "dbStorage_directIOEntryLoggerTotalWriteBufferSizeMb"; + public static final String DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB = + "dbStorage_directIOEntryLoggerTotalReadBufferSizeMb"; + public static final String DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB = + "dbStorage_directIOEntryLoggerReadBufferSizeMb"; + public static final String DIRECT_IO_ENTRYLOGGER_MAX_READ_FDS_PER_THREAD = + "dbStorage_directIOEntryLoggerMaxReadFdsPerThread"; + public static final String DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS = + "dbStorage_directIOEntryLoggerMaxFdCacheTimeSeconds"; static final String MAX_THROTTLE_TIME_MILLIS = "dbStorage_maxThrottleTimeMs"; @@ -88,9 +106,18 @@ public class DbLedgerStorage implements LedgerStorage { private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) / MB; - static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize"; private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100; + private static final long DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB = + (long) (0.125 * PlatformDependent.maxDirectMemory()) + / MB; + private static final long DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB = + (long) (0.125 * PlatformDependent.maxDirectMemory()) + / MB; + private static final long DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB = 8; + + private static final int DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS = 300; + // use the storage assigned to ledger 0 for flags. // if the storage configuration changes, the flags may be lost // but in that case data integrity should kick off anyhow. @@ -100,6 +127,9 @@ public class DbLedgerStorage implements LedgerStorage { // Keep 1 single Bookie GC thread so the the compactions from multiple individual directories are serialized private ScheduledExecutorService gcExecutor; + private ExecutorService entryLoggerWriteExecutor = null; + private ExecutorService entryLoggerFlushExecutor = null; + private DbLedgerStorageStats stats; protected ByteBufAllocator allocator; @@ -127,7 +157,7 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB; long readCacheMaxSize = getLongVariableOrDefault(conf, READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB; - + boolean directIOEntryLogger = getBooleanVariableOrDefault(conf, DIRECT_IO_ENTRYLOGGER, false); this.allocator = allocator; this.numberOfDirs = ledgerDirsManager.getAllLedgerDirs().size(); @@ -153,9 +183,55 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le // Remove the `/current` suffix which will be appended again by LedgersDirManager dirs[0] = ledgerDir.getParentFile(); LedgerDirsManager ldm = new LedgerDirsManager(conf, dirs, ledgerDirsManager.getDiskChecker(), statsLogger); - ledgerStorageList.add(newSingleDirectoryDbLedgerStorage(conf, ledgerManager, ldm, indexDirsManager, - statsLogger, gcExecutor, perDirectoryWriteCacheSize, - perDirectoryReadCacheSize, readAheadCacheBatchSize)); + + EntryLogger entrylogger; + if (directIOEntryLogger) { + long perDirectoryTotalWriteBufferSize = MB * getLongVariableOrDefault( + conf, + DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB, + DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB) / numberOfDirs; + long perDirectoryTotalReadBufferSize = MB * getLongVariableOrDefault( + conf, + DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB, + DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB) / numberOfDirs; + int readBufferSize = MB * (int) getLongVariableOrDefault( + conf, + DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB, + DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB); + int maxFdCacheTimeSeconds = (int) getLongVariableOrDefault( + conf, + DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS, + DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS); + Slf4jSlogger slog = new Slf4jSlogger(DbLedgerStorage.class); + entryLoggerWriteExecutor = Executors.newSingleThreadExecutor( + new DefaultThreadFactory("EntryLoggerWrite")); + entryLoggerFlushExecutor = Executors.newSingleThreadExecutor( + new DefaultThreadFactory("EntryLoggerFlush")); + + int numReadThreads = conf.getNumReadWorkerThreads(); + if (numReadThreads == 0) { + numReadThreads = conf.getServerNumIOThreads(); + } + + entrylogger = new DirectEntryLogger(ledgerDir, new EntryLogIdsImpl(ledgerDirsManager, slog), + new NativeIOImpl(), + allocator, entryLoggerWriteExecutor, entryLoggerFlushExecutor, + conf.getEntryLogSizeLimit(), + conf.getNettyMaxFrameSizeBytes() - 500, + perDirectoryTotalWriteBufferSize, + perDirectoryTotalReadBufferSize, + readBufferSize, + numReadThreads, + maxFdCacheTimeSeconds, + slog, statsLogger); + } else { + entrylogger = new DefaultEntryLogger(conf, ldm, null, statsLogger, allocator); + } + ledgerStorageList.add(newSingleDirectoryDbLedgerStorage(conf, ledgerManager, ldm, + indexDirsManager, entrylogger, + statsLogger, gcExecutor, perDirectoryWriteCacheSize, + perDirectoryReadCacheSize, + readAheadCacheBatchSize)); ldm.getListeners().forEach(ledgerDirsManager::addLedgerDirsListener); } @@ -190,10 +266,11 @@ public Long getSample() { @VisibleForTesting protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, + EntryLogger entryLogger, StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { - return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, + return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, entryLogger, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } @@ -221,6 +298,12 @@ public void shutdown() throws InterruptedException { for (LedgerStorage ls : ledgerStorageList) { ls.shutdown(); } + if (entryLoggerWriteExecutor != null) { + entryLoggerWriteExecutor.shutdown(); + } + if (entryLoggerFlushExecutor != null) { + entryLoggerFlushExecutor.shutdown(); + } } @Override @@ -432,6 +515,19 @@ static long getLongVariableOrDefault(ServerConfiguration conf, String keyName, l } } + static boolean getBooleanVariableOrDefault(ServerConfiguration conf, String keyName, boolean defaultValue) { + Object obj = conf.getProperty(keyName); + if (obj instanceof Boolean) { + return (Boolean) obj; + } else if (obj == null) { + return defaultValue; + } else if (StringUtils.isEmpty(conf.getString(keyName))) { + return defaultValue; + } else { + return conf.getBoolean(keyName); + } + } + @Override public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { // check Issue #2078 diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 30516f82c67..4e72a2ebddd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -146,7 +146,8 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage private final Counter flushExecutorTime; public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, - LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger, + LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, + EntryLogger entryLogger, StatsLogger statsLogger, ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { checkArgument(ledgerDirsManager.getAllLedgerDirs().size() == 1, @@ -186,8 +187,9 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TimeUnit.MINUTES); - entryLogger = new DefaultEntryLogger(conf, ledgerDirsManager, null, statsLogger, allocator); - gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, entryLogger, statsLogger); + this.entryLogger = entryLogger; + gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, entryLogger, + statsLogger); dbLedgerStorageStats = new DbLedgerStorageStats( ledgerDirStatsLogger, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java index 923cf1c1ae6..b8bd64bc7e9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/GarbageCollectorThreadTest.java @@ -22,6 +22,7 @@ import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.logIdFromLocation; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirectEntryLogger; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirsManager; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newLegacyEntryLogger; import static org.hamcrest.MatcherAssert.assertThat; @@ -152,6 +153,14 @@ public void testExtractMetaFromEntryLogsLegacy() throws Exception { newLegacyEntryLogger(20000, ledgerDir), ledgerDir); } + @Test + public void testExtractMetaFromEntryLogsDirect() throws Exception { + File ledgerDir = tmpDirs.createNew("testExtractMeta", "ledgers"); + testExtractMetaFromEntryLogs( + newDirectEntryLogger(23000, // direct header is 4kb rather than 1kb + ledgerDir), ledgerDir); + } + private void testExtractMetaFromEntryLogs(EntryLogger entryLogger, File ledgerDir) throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java index f6ed19f6fed..6be2ac22cc9 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java @@ -24,6 +24,7 @@ import io.netty.buffer.ByteBufAllocator; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; import java.util.List; @@ -58,6 +59,7 @@ private static class LedgerInfo { private final ConcurrentHashMap ledgers = new ConcurrentHashMap<>(); private final EnumSet storageStateFlags = EnumSet.noneOf(StorageState.class); + private final List entryLocations = new ArrayList<>(); @Override public void initialize(ServerConfiguration conf, @@ -301,14 +303,17 @@ public Iterable getActiveLedgersInRange(long firstLedgerId, long lastLedge throw new UnsupportedOperationException("Not supported in mock, implement if you need it"); } - @Override - public void updateEntriesLocations(Iterable locations) throws IOException { - throw new UnsupportedOperationException("Not supported in mock, implement if you need it"); + public List getUpdatedLocations() { + return entryLocations; } @Override - public void flushEntriesLocationsIndex() throws IOException { - throw new UnsupportedOperationException("Not supported in mock, implement if you need it"); + public void updateEntriesLocations(Iterable locations) throws IOException { + synchronized (entryLocations) { + for (EntryLocation l : locations) { + entryLocations.add(l); + } + } } @Override @@ -325,4 +330,7 @@ public void setStorageStateFlag(StorageState flag) throws IOException { public void clearStorageStateFlag(StorageState flag) throws IOException { storageStateFlags.remove(flag); } + + @Override + public void flushEntriesLocationsIndex() throws IOException { } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java index 22e607a2236..ca6d75809f4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java @@ -20,6 +20,10 @@ */ package org.apache.bookkeeper.bookie.storage; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.util.concurrent.MoreExecutors; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; @@ -27,6 +31,8 @@ import java.util.Arrays; import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.LedgerDirsManager; +import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; @@ -50,17 +56,47 @@ public static EntryLogger newLegacyEntryLogger(int logSizeLimit, File... ledgerD NullStatsLogger.INSTANCE, ByteBufAllocator.DEFAULT); } + public static DirectEntryLogger newDirectEntryLogger(int logSizeLimit, File ledgerDir) throws Exception { + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + return new DirectEntryLogger( + curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + logSizeLimit, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE); + } + public static int logIdFromLocation(long location) { return (int) (location >> 32); } public static ByteBuf makeEntry(long ledgerId, long entryId, int size) { + return makeEntry(ledgerId, entryId, size, (byte) 0xdd); + } + + public static ByteBuf makeEntry(long ledgerId, long entryId, int size, byte pattern) { ByteBuf buf = Unpooled.buffer(size); buf.writeLong(ledgerId).writeLong(entryId); - byte[] random = new byte[buf.writableBytes()]; - Arrays.fill(random, (byte) 0xdd); - buf.writeBytes(random); + byte[] data = new byte[buf.writableBytes()]; + Arrays.fill(data, pattern); + buf.writeBytes(data); return buf; } + + public static void assertEntryEquals(ByteBuf e1, ByteBuf e2) throws Exception { + assertThat(e1.readableBytes(), equalTo(e2.readableBytes())); + assertThat(e1, equalTo(e2)); + } + } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/MockEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/MockEntryLogIds.java new file mode 100644 index 00000000000..9e350e68693 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/MockEntryLogIds.java @@ -0,0 +1,34 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * MockEntryLogIds. + */ +public class MockEntryLogIds implements EntryLogIds { + private final AtomicInteger counter = new AtomicInteger(0); + @Override + public int nextId() { + return counter.incrementAndGet(); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java new file mode 100644 index 00000000000..cf73de46229 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java @@ -0,0 +1,246 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage; + +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.logIdFromLocation; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirsManager; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newLegacyEntryLogger; +import static org.apache.commons.lang3.ArrayUtils.toObject; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.arrayContaining; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +import com.google.common.collect.Lists; + +import io.netty.buffer.ByteBuf; +import java.io.File; + +import org.apache.bookkeeper.bookie.EntryLogger; +import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectCompactionEntryLog; +import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Test; + +/** + * TestEntryLogIds. + */ +public class TestEntryLogIds { + private static final Slogger slog = Slogger.CONSOLE; + + private final TmpDirs tmpDirs = new TmpDirs(); + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + } + + @Test + public void testNoStomping() throws Exception { + File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); + + int highestSoFar = -1; + try (EntryLogger legacy = newLegacyEntryLogger(1024, ledgerDir)) { + ByteBuf e1 = makeEntry(1L, 1L, 2048); + long loc1 = legacy.addEntry(1L, e1); + int logId1 = logIdFromLocation(loc1); + + ByteBuf e2 = makeEntry(1L, 2L, 2048); + long loc2 = legacy.addEntry(1L, e2); + int logId2 = logIdFromLocation(loc2); + assertThat(logId2, greaterThan(logId1)); + highestSoFar = logId2; + } + + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId3 = ids.nextId(); + assertThat(logId3, greaterThan(highestSoFar)); + touchLog(ledgerDir, logId3); + highestSoFar = logId3; + + int logId4 = ids.nextId(); + assertThat(logId4, greaterThan(highestSoFar)); + touchLog(ledgerDir, logId4); + highestSoFar = logId4; + + try (EntryLogger legacy = newLegacyEntryLogger(1024, ledgerDir)) { + ByteBuf e1 = makeEntry(1L, 1L, 2048); + long loc5 = legacy.addEntry(1L, e1); + int logId5 = logIdFromLocation(loc5); + assertThat(logId5, greaterThan(highestSoFar)); + + ByteBuf e2 = makeEntry(1L, 2L, 2048); + long loc6 = legacy.addEntry(1L, e2); + int logId6 = logIdFromLocation(loc6); + assertThat(logId6, greaterThan(logId5)); + } + } + + @Test + public void testNoStompingDirectStartsFirst() throws Exception { + File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); + + int highestSoFar = -1; + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId1 = ids.nextId(); + assertThat(logId1, greaterThan(highestSoFar)); + touchLog(ledgerDir, logId1); + highestSoFar = logId1; + + try (EntryLogger legacy = newLegacyEntryLogger(1024, ledgerDir)) { + ByteBuf e1 = makeEntry(1L, 1L, 2048); + long loc2 = legacy.addEntry(1L, e1); + int logId2 = logIdFromLocation(loc2); + assertThat(logId2, greaterThan(highestSoFar)); + highestSoFar = logId2; + + ByteBuf e2 = makeEntry(1L, 2L, 2048); + long loc3 = legacy.addEntry(1L, e2); + int logId3 = logIdFromLocation(loc3); + assertThat(logId3, greaterThan(logId2)); + highestSoFar = logId3; + } + + // reinitialize to pick up legacy + ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId4 = ids.nextId(); + assertThat(logId4, greaterThan(highestSoFar)); + touchLog(ledgerDir, logId4); + highestSoFar = logId4; + } + + @Test + public void testMultiDirectory() throws Exception { + File base = tmpDirs.createNew("entryLogIds", "ledgers"); + File ledgerDir1 = new File(base, "l1"); + File ledgerDir2 = new File(base, "l2"); + File ledgerDir3 = new File(base, "l3"); + + int highestSoFar = -1; + try (EntryLogger legacy = newLegacyEntryLogger(1024, ledgerDir1, ledgerDir2, ledgerDir3)) { + ByteBuf e1 = makeEntry(1L, 1L, 2048); + long loc1 = legacy.addEntry(1L, e1); + int logId1 = logIdFromLocation(loc1); + assertThat(logId1, greaterThan(highestSoFar)); + highestSoFar = logId1; + + ByteBuf e2 = makeEntry(1L, 2L, 2048); + long loc2 = legacy.addEntry(1L, e2); + int logId2 = logIdFromLocation(loc2); + assertThat(logId2, greaterThan(highestSoFar)); + highestSoFar = logId2; + + ByteBuf e3 = makeEntry(1L, 3L, 2048); + long loc3 = legacy.addEntry(1L, e3); + int logId3 = logIdFromLocation(loc3); + assertThat(logId3, greaterThan(highestSoFar)); + highestSoFar = logId3; + } + + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir1, ledgerDir2, ledgerDir3), slog); + int logId4 = ids.nextId(); + assertThat(logId4, greaterThan(highestSoFar)); + touchLog(ledgerDir2, logId4); + highestSoFar = logId4; + + try (EntryLogger legacy = newLegacyEntryLogger(1024, ledgerDir1, ledgerDir2, ledgerDir3)) { + ByteBuf e1 = makeEntry(1L, 1L, 2048); + long loc5 = legacy.addEntry(1L, e1); + int logId5 = logIdFromLocation(loc5); + assertThat(logId5, greaterThan(highestSoFar)); + highestSoFar = logId5; + } + } + + @Test + public void testWrapAround() throws Exception { + File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); + new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + touchLog(ledgerDir, Integer.MAX_VALUE - 1); + + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId = ids.nextId(); + assertThat(logId, equalTo(0)); + } + + @Test + public void testCompactingLogsNotConsidered() throws Exception { + // if there is a process restart, all "compacting" logs will be deleted + // so their IDs are safe to reuse. Even in the case of two processes acting + // the directory concurrently, the transactional rename will prevent data + // loss. + File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); + new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + touchLog(ledgerDir, 123); + touchCompacting(ledgerDir, 129); + + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId = ids.nextId(); + assertThat(logId, equalTo(124)); + } + + @Test + public void testCompactedLogsConsidered() throws Exception { + File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); + new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + touchLog(ledgerDir, 123); + touchCompacted(ledgerDir, 129, 123); + + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + int logId = ids.nextId(); + assertThat(logId, equalTo(130)); + } + + + @Test + public void testGapSelection() throws Exception { + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList())), + arrayContaining(0, Integer.MAX_VALUE)); + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(0))), + arrayContaining(1, Integer.MAX_VALUE)); + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(1, 2, 3, 4, 5, 6))), + arrayContaining(7, Integer.MAX_VALUE)); + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE))), + arrayContaining(0, Integer.MAX_VALUE)); + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2))), + arrayContaining(0, Integer.MAX_VALUE / 2)); + assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2 - 1))), + arrayContaining(Integer.MAX_VALUE / 2, Integer.MAX_VALUE)); + } + + private static void touchLog(File ledgerDir, int logId) throws Exception { + assertThat(DirectEntryLogger.logFile(new File(ledgerDir, "current"), logId).createNewFile(), + equalTo(true)); + } + + private static void touchCompacting(File ledgerDir, int logId) throws Exception { + assertThat(DirectCompactionEntryLog.compactingFile(new File(ledgerDir, "current"), logId).createNewFile(), + equalTo(true)); + } + + private static void touchCompacted(File ledgerDir, int newLogId, int compactedLogId) throws Exception { + assertThat(DirectCompactionEntryLog.compactedFile(new File(ledgerDir, "current"), newLogId, compactedLogId) + .createNewFile(), equalTo(true)); + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java new file mode 100644 index 00000000000..748acd6d16e --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java @@ -0,0 +1,515 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.assertEntryEquals; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.logIdFromLocation; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.storage.EntryLogger; +import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +/** + * TestDirectEntryLogger. + */ +public class TestDirectEntryLogger { + private final Slogger slog = Slogger.CONSOLE; + + private static final long ledgerId1 = 1234; + + private final TmpDirs tmpDirs = new TmpDirs(); + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + } + + @Test + public void testLogRolling() throws Exception { + File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 4000); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 4000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 4000); + + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + long loc1 = elog.addEntry(ledgerId1, e1.slice()); + int logId1 = logIdFromLocation(loc1); + assertThat(logId1, equalTo(1)); + + long loc2 = elog.addEntry(ledgerId1, e2.slice()); + int logId2 = logIdFromLocation(loc2); + assertThat(logId2, equalTo(2)); + + long loc3 = elog.addEntry(ledgerId1, e3.slice()); + int logId3 = logIdFromLocation(loc3); + assertThat(logId3, equalTo(3)); + } + } + + @Test + public void testReadLog() throws Exception { + File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 100); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 100); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 100); + + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 200000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + long loc1 = elog.addEntry(ledgerId1, e1.slice()); + long loc2 = elog.addEntry(ledgerId1, e2.slice()); + elog.flush(); + + ByteBuf e1read = elog.readEntry(ledgerId1, 1L, loc1); + ByteBuf e2read = elog.readEntry(ledgerId1, 2L, loc2); + assertEntryEquals(e1read, e1); + assertEntryEquals(e2read, e2); + e1read.release(); + e2read.release(); + + long loc3 = elog.addEntry(ledgerId1, e3.slice()); + elog.flush(); + + ByteBuf e3read = elog.readEntry(ledgerId1, 3L, loc3); + assertEntryEquals(e3read, e3); + e3read.release(); + } + } + + @Test + public void testLogReaderCleanup() throws Exception { + File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + final int entrySize = Buffer.ALIGNMENT; + final int maxFileSize = Header.EMPTY_HEADER.length + entrySize; + final int maxCachedReaders = 16; + + AtomicInteger outstandingReaders = new AtomicInteger(0); + EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + maxFileSize, + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + maxCachedReaders * maxFileSize, // total read buffer size + maxFileSize, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE) { + @Override + LogReader newDirectReader(int logId) throws IOException { + outstandingReaders.incrementAndGet(); + return new DirectReader(logId, logFilename(curDir, logId), ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, 10 * 1024 * 1024, + NullStatsLogger.INSTANCE.getOpStatsLogger("")) { + @Override + public void close() throws IOException { + super.close(); + outstandingReaders.decrementAndGet(); + } + }; + } + }; + try { + List locations = new ArrayList<>(); + // `+ 1` is not a typo: create one more log file than the max number of o cached readers + for (int i = 0; i < maxCachedReaders + 1; i++) { + ByteBuf e = makeEntry(ledgerId1, i, entrySize); + long loc = elog.addEntry(ledgerId1, e.slice()); + locations.add(loc); + } + elog.flush(); + for (Long loc : locations) { + elog.readEntry(loc).release(); + } + assertThat(outstandingReaders.get(), equalTo(maxCachedReaders)); + } finally { + elog.close(); + } + assertThat(outstandingReaders.get(), equalTo(0)); + } + + @Test + public void testReadMetadataAndScan() throws Exception { + File ledgerDir = tmpDirs.createNew("directCanReadAndScanMeta", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long ledgerId1 = 1L; + long ledgerId2 = 2L; + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId2, 2L, 2000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 3000); + + long loc1, loc2, loc3; + try (DirectEntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 16, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1); + loc2 = elog.addEntry(ledgerId2, e2); + loc3 = elog.addEntry(ledgerId1, e3); + } + + try (DirectEntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 16, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + int logId = logIdFromLocation(loc1); + assertThat(logId, equalTo(logIdFromLocation(loc2))); + assertThat(logId, equalTo(logIdFromLocation(loc3))); + + EntryLogMetadata metaRead = elog.readEntryLogIndex(logId); + assertThat(metaRead.getEntryLogId(), equalTo((long) logId)); + assertThat(metaRead.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(metaRead.getRemainingSize(), equalTo(metaRead.getTotalSize())); + assertThat(metaRead.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(metaRead.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + + EntryLogMetadata metaScan = elog.scanEntryLogMetadata(logId, null); + assertThat(metaScan.getEntryLogId(), equalTo((long) logId)); + assertThat(metaScan.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(metaScan.getRemainingSize(), equalTo(metaScan.getTotalSize())); + assertThat(metaScan.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(metaScan.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + } + } + + @Test + public void testMetadataFallback() throws Exception { + File ledgerDir = tmpDirs.createNew("directMetaFallback", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long ledgerId1 = 1L; + long ledgerId2 = 2L; + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId2, 2L, 2000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 3000); + + int maxFileSize = 1000 + 2000 + 3000 + (Integer.BYTES * 3) + 4096; + long loc1, loc2, loc3; + try (DirectEntryLogger writer = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 16, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = writer.addEntry(ledgerId1, e1); + loc2 = writer.addEntry(ledgerId2, e2); + loc3 = writer.addEntry(ledgerId1, e3); + writer.flush(); + + try (DirectEntryLogger reader = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 16, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + int logId = logIdFromLocation(loc1); + try { + reader.readEntryLogIndex(logId); + Assert.fail("Shouldn't be there"); + } catch (IOException ioe) { + // expected + } + + EntryLogMetadata metaRead = reader.getEntryLogMetadata(logId); // should fail read, fallback to scan + assertThat(metaRead.getEntryLogId(), equalTo((long) logId)); + assertThat(metaRead.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(metaRead.getRemainingSize(), equalTo(metaRead.getTotalSize())); + assertThat(metaRead.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(metaRead.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + } + } + } + + @Test + public void testMetadataManyBatch() throws Exception { + File ledgerDir = tmpDirs.createNew("directMetaManyBatches", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long lastLoc = -1; + int ledgerCount = 11000; + try (DirectEntryLogger writer = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 24, // max file size + 10 * 1024 * 1024, // max sane entry size + 32 * 1024 * 1024, // total write buffer size + 32 * 1024 * 1024, // total read buffer size + 16 * 1024 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + for (int i = 0; i < ledgerCount; i++) { + long loc = writer.addEntry(i, makeEntry(i, 1L, 1000)); + if (lastLoc >= 0) { + assertThat(logIdFromLocation(loc), equalTo(logIdFromLocation(lastLoc))); + } + lastLoc = loc; + } + writer.flush(); + } + + try (DirectEntryLogger reader = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 20, // max file size + 10 * 1024 * 1024, // max sane entry size + 32 * 1024 * 1024, // total write buffer size + 32 * 1024 * 1024, // total read buffer size + 16 * 1024 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + int logId = logIdFromLocation(lastLoc); + EntryLogMetadata metaRead = reader.readEntryLogIndex(logId); + + assertThat(metaRead.getEntryLogId(), equalTo((long) logId)); + assertThat(metaRead.getTotalSize(), equalTo((1000L + Integer.BYTES) * ledgerCount)); + assertThat(metaRead.getRemainingSize(), equalTo(metaRead.getTotalSize())); + for (int i = 0; i < ledgerCount; i++) { + assertThat(metaRead.getLedgersMap().containsKey(i), equalTo(true)); + } + } + } + + @Test + public void testGetFlushedLogs() throws Exception { + File ledgerDir = tmpDirs.createNew("testFlushedLogs", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ExecutorService executor = Executors.newFixedThreadPool(5); + CompletableFuture blockClose = new CompletableFuture<>(); + NativeIOImpl nativeIO = new NativeIOImpl() { + @Override + public int close(int fd) { + try { + blockClose.join(); + return super.close(fd); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + }; + DirectEntryLogger entryLogger = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + nativeIO, + ByteBufAllocator.DEFAULT, + executor, + executor, + 23000, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 32 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE); + try { // not using try-with-resources because close needs to be unblocked in failure + // Add entries. + // Ledger 1 is on first entry log + // Ledger 2 spans first, second and third entry log + // Ledger 3 is on the third entry log (which is still active when extract meta) + long loc1 = entryLogger.addEntry(1L, makeEntry(1L, 1L, 5000)); + long loc2 = entryLogger.addEntry(2L, makeEntry(2L, 1L, 5000)); + assertThat(logIdFromLocation(loc2), equalTo(logIdFromLocation(loc1))); + long loc3 = entryLogger.addEntry(2L, makeEntry(2L, 1L, 15000)); + assertThat(logIdFromLocation(loc3), greaterThan(logIdFromLocation(loc2))); + long loc4 = entryLogger.addEntry(2L, makeEntry(2L, 1L, 15000)); + assertThat(logIdFromLocation(loc4), greaterThan(logIdFromLocation(loc3))); + long loc5 = entryLogger.addEntry(3L, makeEntry(3L, 1L, 1000)); + assertThat(logIdFromLocation(loc5), equalTo(logIdFromLocation(loc4))); + + long logId1 = logIdFromLocation(loc2); + long logId2 = logIdFromLocation(loc3); + long logId3 = logIdFromLocation(loc5); + + // all three should exist + assertThat(entryLogger.logExists(logId1), equalTo(true)); + assertThat(entryLogger.logExists(logId2), equalTo(true)); + assertThat(entryLogger.logExists(logId3), equalTo(true)); + + assertThat(entryLogger.getFlushedLogIds(), empty()); + + blockClose.complete(null); + entryLogger.flush(); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2)); + + long loc6 = entryLogger.addEntry(3L, makeEntry(3L, 1L, 25000)); + assertThat(logIdFromLocation(loc6), greaterThan(logIdFromLocation(loc5))); + entryLogger.flush(); + + assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2, logId3)); + } finally { + blockClose.complete(null); + entryLogger.close(); + executor.shutdownNow(); + } + } + + @Test + public void testBufferSizeNotPageAligned() throws Exception { + File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 4000); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 4000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 4000); + + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 128 * 1024 + 500, // total write buffer size + 128 * 1024 + 300, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + long loc1 = elog.addEntry(ledgerId1, e1.slice()); + int logId1 = logIdFromLocation(loc1); + assertThat(logId1, equalTo(1)); + + long loc2 = elog.addEntry(ledgerId1, e2.slice()); + int logId2 = logIdFromLocation(loc2); + assertThat(logId2, equalTo(2)); + + long loc3 = elog.addEntry(ledgerId1, e3.slice()); + int logId3 = logIdFromLocation(loc3); + assertThat(logId3, equalTo(3)); + } + } +} + diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java new file mode 100644 index 00000000000..487e94a0711 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java @@ -0,0 +1,329 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.assertEntryEquals; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.logIdFromLocation; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newLegacyEntryLogger; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +import com.google.common.util.concurrent.MoreExecutors; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import java.io.File; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.storage.EntryLogger; +import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Test; + +/** + * TestDirectEntryLoggerCompat. + */ +public class TestDirectEntryLoggerCompat { + private final Slogger slog = Slogger.CONSOLE; + + private static final long ledgerId1 = 1234; + private static final long ledgerId2 = 4567; + private static final long ledgerId3 = 7890; + + private final TmpDirs tmpDirs = new TmpDirs(); + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + } + + @Test + public void testLegacyCanReadDirect() throws Exception { + File ledgerDir = tmpDirs.createNew("legacyCanRead", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 1000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 1000); + + long loc1, loc2, loc3; + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 10 * 1024 * 1024, // 10MiB, max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1.slice()); + loc2 = elog.addEntry(ledgerId1, e2.slice()); + loc3 = elog.addEntry(ledgerId1, e3.slice()); + } + + try (EntryLogger legacy = newLegacyEntryLogger(2000000, ledgerDir)) { + assertEntryEquals(legacy.readEntry(ledgerId1, 1L, loc1), e1); + assertEntryEquals(legacy.readEntry(ledgerId1, 2L, loc2), e2); + assertEntryEquals(legacy.readEntry(ledgerId1, 3L, loc3), e3); + } + } + + @Test + public void testDirectCanReadLegacy() throws Exception { + File ledgerDir = tmpDirs.createNew("legacyCanRead", "ledgers"); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 1000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 1000); + + long loc1, loc2, loc3; + try (EntryLogger legacy = newLegacyEntryLogger(2000000, ledgerDir)) { + loc1 = legacy.addEntry(ledgerId1, e1.slice()); + loc2 = legacy.addEntry(ledgerId1, e2.slice()); + loc3 = legacy.addEntry(ledgerId1, e3.slice()); + legacy.flush(); + } + + try (EntryLogger elog = new DirectEntryLogger( + new File(ledgerDir, "current"), new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 10 * 1024 * 1024, // 10MiB, max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + assertEntryEquals(elog.readEntry(ledgerId1, 1L, loc1), e1); + assertEntryEquals(elog.readEntry(ledgerId1, 2L, loc2), e2); + assertEntryEquals(elog.readEntry(ledgerId1, 3L, loc3), e3); + } + } + + @Test + public void testLegacyCanReadDirectAfterMultipleRolls() throws Exception { + File ledgerDir = tmpDirs.createNew("legacyCanRead", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 4000); + ByteBuf e2 = makeEntry(ledgerId1, 2L, 4000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 4000); + + long loc1, loc2, loc3; + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 6000, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1.slice()); + loc2 = elog.addEntry(ledgerId1, e2.slice()); + loc3 = elog.addEntry(ledgerId1, e3.slice()); + } + + try (EntryLogger legacy = newLegacyEntryLogger(2000000, ledgerDir)) { + assertEntryEquals(legacy.readEntry(ledgerId1, 1L, loc1), e1); + assertEntryEquals(legacy.readEntry(ledgerId1, 2L, loc2), e2); + assertEntryEquals(legacy.readEntry(ledgerId1, 3L, loc3), e3); + } + } + + @Test + public void testLegacyCanReadMetadataOfDirectWithIndexWritten() throws Exception { + File ledgerDir = tmpDirs.createNew("legacyCanReadMeta", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId2, 2L, 2000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 3000); + ByteBuf e4 = makeEntry(ledgerId1, 4L, 4000); + + int maxFileSize = 1000 + 2000 + 3000 + (Integer.BYTES * 3) + 4096; + long loc1, loc2, loc3, loc4; + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + maxFileSize, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1); + loc2 = elog.addEntry(ledgerId2, e2); + loc3 = elog.addEntry(ledgerId1, e3); + loc4 = elog.addEntry(ledgerId1, e4); + } + + try (EntryLogger legacy = newLegacyEntryLogger( + maxFileSize, // size of first 3 entries + header + ledgerDir)) { + int logId = logIdFromLocation(loc1); + assertThat(logId, equalTo(logIdFromLocation(loc2))); + assertThat(logId, equalTo(logIdFromLocation(loc3))); + assertThat(logId, not(equalTo(logIdFromLocation(loc4)))); + + EntryLogMetadata meta = legacy.getEntryLogMetadata(logId); + + assertThat(meta.getEntryLogId(), equalTo((long) logId)); + assertThat(meta.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + assertThat(meta.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(meta.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + } + } + + @Test + public void testLegacyCanReadMetadataOfDirectWithNoIndexWritten() throws Exception { + File ledgerDir = tmpDirs.createNew("legacyCanReadMeta", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId2, 2L, 2000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 3000); + ByteBuf e4 = makeEntry(ledgerId1, 4L, 4000); + + int maxFileSize = 1000 + 2000 + 3000 + (Integer.BYTES * 3) + 4096; + long loc1, loc2, loc3; + try (EntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + maxFileSize * 10, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1); + loc2 = elog.addEntry(ledgerId2, e2); + loc3 = elog.addEntry(ledgerId1, e3); + } + + try (EntryLogger legacy = newLegacyEntryLogger( + maxFileSize, // size of first 3 entries + header + ledgerDir)) { + int logId = logIdFromLocation(loc1); + assertThat(logId, equalTo(logIdFromLocation(loc2))); + assertThat(logId, equalTo(logIdFromLocation(loc3))); + EntryLogMetadata meta = legacy.getEntryLogMetadata(logId); + + assertThat(meta.getEntryLogId(), equalTo((long) logId)); + assertThat(meta.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + assertThat(meta.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(meta.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + } + } + + @Test + public void testDirectCanReadMetadataAndScanFromLegacy() throws Exception { + File ledgerDir = tmpDirs.createNew("directCanReadLegacyMeta", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + ByteBuf e1 = makeEntry(ledgerId1, 1L, 1000); + ByteBuf e2 = makeEntry(ledgerId2, 2L, 2000); + ByteBuf e3 = makeEntry(ledgerId1, 3L, 3000); + ByteBuf e4 = makeEntry(ledgerId1, 4L, 4000); + + int maxFileSize = 1000 + 2000 + 3000 + (Integer.BYTES * 3) + 4096; + long loc1, loc2, loc3, loc4; + try (EntryLogger legacy = newLegacyEntryLogger( + maxFileSize, // size of first 3 entries + header + ledgerDir)) { + loc1 = legacy.addEntry(ledgerId1, e1); + loc2 = legacy.addEntry(ledgerId2, e2); + loc3 = legacy.addEntry(ledgerId1, e3); + loc4 = legacy.addEntry(ledgerId1, e4); // should force a roll + } + + try (DirectEntryLogger elog = new DirectEntryLogger( + curDir, new MockEntryLogIds(), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + maxFileSize * 10, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + int logId = logIdFromLocation(loc1); + assertThat(logId, equalTo(logIdFromLocation(loc2))); + assertThat(logId, equalTo(logIdFromLocation(loc3))); + assertThat(logId, not(equalTo(logIdFromLocation(loc4)))); + + EntryLogMetadata metaRead = elog.readEntryLogIndex(logId); + assertThat(metaRead.getEntryLogId(), equalTo((long) logId)); + assertThat(metaRead.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(metaRead.getRemainingSize(), equalTo(metaRead.getTotalSize())); + assertThat(metaRead.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(metaRead.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + + EntryLogMetadata metaScan = elog.scanEntryLogMetadata(logId, null); + assertThat(metaScan.getEntryLogId(), equalTo((long) logId)); + assertThat(metaScan.getTotalSize(), equalTo(1000L + 2000 + 3000 + (Integer.BYTES * 3))); + assertThat(metaScan.getRemainingSize(), equalTo(metaScan.getTotalSize())); + assertThat(metaScan.getLedgersMap().get(ledgerId1), equalTo(1000L + 3000L + (Integer.BYTES * 2))); + assertThat(metaScan.getLedgersMap().get(ledgerId2), equalTo(2000L + Integer.BYTES)); + } + } + +} + diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java new file mode 100644 index 00000000000..e7d810f99b0 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java @@ -0,0 +1,509 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger.logFilename; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.util.concurrent.MoreExecutors; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.apache.bookkeeper.common.util.nativeio.NativeIOException; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.stats.OpStatsLogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + + +/** + * TestDirectReader. + */ +public class TestDirectReader { + private static final Slogger slog = Slogger.CONSOLE; + + private final TmpDirs tmpDirs = new TmpDirs(); + private final ExecutorService writeExecutor = Executors.newSingleThreadExecutor(); + private final OpStatsLogger opLogger = NullStatsLogger.INSTANCE.getOpStatsLogger("null"); + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + writeExecutor.shutdownNow(); + } + + @Test + public void testReadInt() throws Exception { + File ledgerDir = tmpDirs.createNew("readInt", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xdeadbeef, 0, 1 << 20); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + assertThat(reader.readIntAt(0), equalTo(0xdeadbeef)); + assertThat(reader.readIntAt(2), equalTo(0xbeefdead)); + assertThat(reader.readIntAt(1024), equalTo(0xdeadbeef)); + assertThat(reader.readIntAt(1025), equalTo(0xadbeefde)); + } + } + + @Test + public void testReadIntAcrossBoundary() throws Exception { + File ledgerDir = tmpDirs.createNew("readInt", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xdeadbeef, 0, 1 << 20); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + assertThat(reader.readIntAt(Buffer.ALIGNMENT - 2), equalTo(0xbeefdead)); + } + } + + @Test + public void testReadLong() throws Exception { + File ledgerDir = tmpDirs.createNew("readLong", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeefcafe, 0, 1 << 20); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + assertThat(reader.readLongAt(0), equalTo(0xbeefcafebeefcafeL)); + assertThat(reader.readLongAt(2), equalTo(0xcafebeefcafebeefL)); + assertThat(reader.readLongAt(1024), equalTo(0xbeefcafebeefcafeL)); + assertThat(reader.readLongAt(1025), equalTo(0xefcafebeefcafebeL)); + } + } + + @Test + public void testReadLongAcrossBoundary() throws Exception { + File ledgerDir = tmpDirs.createNew("readLong", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeefcafe, 0, 1 << 20); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + assertThat(reader.readLongAt(0), equalTo(0xbeefcafebeefcafeL)); + assertThat(reader.readLongAt(2), equalTo(0xcafebeefcafebeefL)); + assertThat(reader.readLongAt(1024), equalTo(0xbeefcafebeefcafeL)); + assertThat(reader.readLongAt(1025), equalTo(0xefcafebeefcafebeL)); + } + } + + @Test + public void testReadBuffer() throws Exception { + File ledgerDir = tmpDirs.createNew("readBuffer", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeefcafe, 1, 1 << 20); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT * 4, + 1 << 20, opLogger)) { + ByteBuf bb = reader.readBufferAt(0, Buffer.ALIGNMENT * 2); + try { + for (int j = 0; j < Buffer.ALIGNMENT / Integer.BYTES; j++) { + assertThat(bb.readInt(), equalTo(0xbeefcafe)); + } + for (int i = 0; i < Buffer.ALIGNMENT / Integer.BYTES; i++) { + assertThat(bb.readInt(), equalTo(0xbeefcaff)); + } + assertThat(bb.readableBytes(), equalTo(0)); + } finally { + bb.release(); + } + + bb = reader.readBufferAt(Buffer.ALIGNMENT * 8, Buffer.ALIGNMENT); + try { + for (int j = 0; j < Buffer.ALIGNMENT / Integer.BYTES; j++) { + assertThat(bb.readInt(), equalTo(0xbeefcb06)); + } + assertThat(bb.readableBytes(), equalTo(0)); + } finally { + bb.release(); + } + + bb = reader.readBufferAt(Buffer.ALIGNMENT * 10 + 123, 345); + try { + assertThat(bb.readByte(), equalTo((byte) 0x08)); + for (int j = 0; j < 344 / Integer.BYTES; j++) { + assertThat(bb.readInt(), equalTo(0xbeefcb08)); + } + assertThat(bb.readableBytes(), equalTo(0)); + } finally { + bb.release(); + } + + } + } + + @Test + public void testReadBufferAcrossBoundary() throws Exception { + File ledgerDir = tmpDirs.createNew("readBuffer", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeefcafe, 1, 1 << 20); + BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT * 4, 8); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT * 4, + 1 << 20, opLogger)) { + ByteBuf bb = reader.readBufferAt((long) (Buffer.ALIGNMENT * 3.5), Buffer.ALIGNMENT); + try { + for (int j = 0; j < (Buffer.ALIGNMENT / Integer.BYTES) / 2; j++) { + assertThat(bb.readInt(), equalTo(0xbeefcb01)); + } + for (int i = 0; i < (Buffer.ALIGNMENT / Integer.BYTES) / 2; i++) { + assertThat(bb.readInt(), equalTo(0xbeefcb02)); + } + assertThat(bb.readableBytes(), equalTo(0)); + } finally { + bb.release(); + } + } + } + + @Test + public void testReadBufferBiggerThanReaderBuffer() throws Exception { + File ledgerDir = tmpDirs.createNew("readBuffer", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeefcafe, 1, 1 << 20); + + // buffer size is ALIGNMENT, read will be ALIGNMENT*2 + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + ByteBuf bb = reader.readBufferAt(0, Buffer.ALIGNMENT * 2); + try { + for (int j = 0; j < Buffer.ALIGNMENT / Integer.BYTES; j++) { + assertThat(bb.readInt(), equalTo(0xbeefcafe)); + } + for (int i = 0; i < Buffer.ALIGNMENT / Integer.BYTES; i++) { + assertThat(bb.readInt(), equalTo(0xbeefcaff)); + } + assertThat(bb.readableBytes(), equalTo(0)); + } finally { + bb.release(); + } + } + } + + @Test(expected = EOFException.class) + public void testReadPastEndOfFile() throws Exception { + File ledgerDir = tmpDirs.createNew("readBuffer", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeeeeeef, 1, 1 << 13); + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + reader.readBufferAt(1 << 13, Buffer.ALIGNMENT); + } + } + + @Test(expected = EOFException.class) + public void testReadPastEndOfFilePartial() throws Exception { + File ledgerDir = tmpDirs.createNew("readBuffer", "logs"); + + writeFileWithPattern(ledgerDir, 1234, 0xbeeeeeef, 1, 1 << 13); + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + reader.readBufferAt((1 << 13) - Buffer.ALIGNMENT / 2, Buffer.ALIGNMENT); + } + } + + @Test + public void testReadEntries() throws Exception { + File ledgerDir = tmpDirs.createNew("readEntries", "logs"); + + int entrySize = Buffer.ALIGNMENT / 4 + 100; + Map offset2Pattern = new HashMap<>(); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), + 1 << 20, MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + for (int i = 0; i < 1000; i++) { + ByteBuf bb = Unpooled.buffer(entrySize); + int pattern = 0xbeef + i; + TestBuffer.fillByteBuf(bb, pattern); + int offset = writer.writeDelimited(bb); + offset2Pattern.put(offset, pattern); + } + } + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + List> offset2PatternList = new ArrayList(offset2Pattern.entrySet()); + Collections.shuffle(offset2PatternList); + + for (Map.Entry e : offset2PatternList) { + ByteBuf entry = reader.readEntryAt(e.getKey()); + try { + assertThat(entry.readableBytes(), equalTo(entrySize)); + while (entry.isReadable()) { + assertThat(entry.readInt(), equalTo(e.getValue())); + } + } finally { + entry.release(); + } + } + } + } + + @Test + public void testReadFromFileBeingWrittenNoPreallocation() throws Exception { + File ledgerDir = tmpDirs.createNew("readWhileWriting", "logs"); + + int entrySize = Buffer.ALIGNMENT / 2 + 8; + NativeIOImpl nativeIO = new NativeIOImpl() { + @Override + public int fallocate(int fd, int mode, long offset, long len) + throws NativeIOException { + return 0; + } + }; + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), + 1 << 20, MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE); + LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + ByteBuf b2 = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(b2, 0xfede); + int offset = writer.writeDelimited(b2); + + try { + reader.readEntryAt(offset); + Assert.fail("Should have failed"); + } catch (IOException ioe) { + // expected + } + writer.flush(); + + ByteBuf bbread = reader.readEntryAt(offset); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.isReadable()) { + assertThat(bbread.readInt(), equalTo(0xfede)); + } + } finally { + bbread.release(); + } + } + } + + @Test + public void testReadFromFileBeingWrittenReadInPreallocated() throws Exception { + File ledgerDir = tmpDirs.createNew("readWhileWriting", "logs"); + + int entrySize = Buffer.ALIGNMENT / 2 + 8; + + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), + 1 << 20, MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE); + LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + ByteBuf bb = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(bb, 0xfeed); + int offset = writer.writeDelimited(bb); + + try { + reader.readEntryAt(offset); + Assert.fail("Should have failed"); + } catch (IOException ioe) { + // expected + } + writer.flush(); + ByteBuf bbread = reader.readEntryAt(offset); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.isReadable()) { + assertThat(bbread.readInt(), equalTo(0xfeed)); + } + } finally { + bbread.release(); + } + } + } + + @Test + public void testPartialRead() throws Exception { + File ledgerDir = tmpDirs.createNew("partialRead", "logs"); + + int entrySize = Buffer.ALIGNMENT * 4; + + NativeIOImpl nativeIO = new NativeIOImpl() { + @Override + public long pread(int fd, long buf, long size, long offset) throws NativeIOException { + long read = super.pread(fd, buf, size, offset); + return Math.min(read, Buffer.ALIGNMENT); // force only less than a buffer read + } + + @Override + public int fallocate(int fd, int mode, long offset, long len) + throws NativeIOException { + return 0; // don't preallocate + } + }; + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT * 10, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, + MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf b1 = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(b1, 0xfeedfeed); + int offset1 = writer.writeDelimited(b1); + + ByteBuf b2 = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(b2, 0xfedefede); + int offset2 = writer.writeDelimited(b2); + writer.flush(); + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + nativeIO, Buffer.ALIGNMENT * 3, + 1 << 20, opLogger)) { + ByteBuf bbread = reader.readEntryAt(offset1); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.readableBytes() >= Integer.BYTES) { + assertThat(bbread.readInt(), equalTo(0xfeedfeed)); + } + assertThat(bbread.readableBytes(), equalTo(0)); + } finally { + bbread.release(); + } + + bbread = reader.readEntryAt(offset2); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.readableBytes() >= Integer.BYTES) { + assertThat(bbread.readInt(), equalTo(0xfedefede)); + } + assertThat(bbread.readableBytes(), equalTo(0)); + } finally { + bbread.release(); + } + } + } + } + + @Test + public void testLargeEntry() throws Exception { + File ledgerDir = tmpDirs.createNew("largeEntries", "logs"); + + int entrySize = Buffer.ALIGNMENT * 4; + + int offset1, offset2; + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT * 8, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, + MoreExecutors.newDirectExecutorService(), buffers, new NativeIOImpl(), + Slogger.CONSOLE)) { + ByteBuf b1 = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(b1, 0xfeedfeed); + offset1 = writer.writeDelimited(b1); + + ByteBuf b2 = Unpooled.buffer(entrySize); + TestBuffer.fillByteBuf(b2, 0xfedefede); + offset2 = writer.writeDelimited(b2); + writer.flush(); + } + + try (LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), + ByteBufAllocator.DEFAULT, + new NativeIOImpl(), Buffer.ALIGNMENT, + 1 << 20, opLogger)) { + ByteBuf bbread = reader.readEntryAt(offset1); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.readableBytes() >= Integer.BYTES) { + assertThat(bbread.readInt(), equalTo(0xfeedfeed)); + } + assertThat(bbread.readableBytes(), equalTo(0)); + } finally { + bbread.release(); + } + + bbread = reader.readEntryAt(offset2); + try { + assertThat(bbread.readableBytes(), equalTo(entrySize)); + while (bbread.readableBytes() >= Integer.BYTES) { + assertThat(bbread.readInt(), equalTo(0xfedefede)); + } + assertThat(bbread.readableBytes(), equalTo(0)); + } finally { + bbread.release(); + } + } + } + + private static void writeFileWithPattern(File directory, int logId, + int pattern, int blockIncrement, int fileSize) throws Exception { + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(logId, logFilename(directory, logId), + fileSize, MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + + for (int written = 0; written < fileSize; written += Buffer.ALIGNMENT) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, pattern); + writer.writeAt(written, bb); + bb.release(); + pattern += blockIncrement; + } + writer.flush(); + } + } + +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectWriter.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectWriter.java new file mode 100644 index 00000000000..aed36ca42de --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectWriter.java @@ -0,0 +1,333 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger.logFilename; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.util.concurrent.MoreExecutors; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import org.apache.bookkeeper.common.util.nativeio.NativeIO; +import org.apache.bookkeeper.common.util.nativeio.NativeIOException; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +/** + * TestDirectWriter. + */ +public class TestDirectWriter { + private static final Slogger slog = Slogger.CONSOLE; + + private final TmpDirs tmpDirs = new TmpDirs(); + private final ExecutorService writeExecutor = Executors.newSingleThreadExecutor(); + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + writeExecutor.shutdownNow(); + } + + @Test(expected = IllegalArgumentException.class) + public void testWriteAtAlignment() throws Exception { + File ledgerDir = tmpDirs.createNew("writeAlignment", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), + 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeAt(1234, bb); + writer.flush(); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testWriteAlignmentSize() throws Exception { + File ledgerDir = tmpDirs.createNew("writeAlignment", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(123); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeAt(0, bb); + writer.flush(); + } + } + + @Test + public void testWriteAlignedNotAtStart() throws Exception { + File ledgerDir = tmpDirs.createNew("writeAlignment", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeAt(Buffer.ALIGNMENT * 2, bb); + writer.flush(); + } + } + + + @Test(timeout = 10000) + public void testFlushingWillWaitForBuffer() throws Exception { + File ledgerDir = tmpDirs.createNew("writeFailFailsFlush", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), + Buffer.ALIGNMENT, 1); // only one buffer available, so we can't flush in bg + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT / 2); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeDelimited(bb); + writer.flush(); + } + } + + @Test(expected = IOException.class) + public void testWriteFailFailsFlush() throws Exception { + File ledgerDir = tmpDirs.createNew("writeFailFailsFlush", "logs"); + NativeIO io = new NativeIOImpl() { + boolean failed = false; + @Override + public int pwrite(int fd, long pointer, int count, long offset) throws NativeIOException { + synchronized (this) { + if (!failed) { + failed = true; + throw new NativeIOException("fail for test"); + } + } + return super.pwrite(fd, pointer, count, offset); + } + }; + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), Buffer.ALIGNMENT, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, io, Slogger.CONSOLE)) { + for (int i = 0; i < 10; i++) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT / 2); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeDelimited(bb); + } + writer.flush(); + } + } + + @Test(expected = IOException.class) + public void testWriteAtFailFailsFlush() throws Exception { + File ledgerDir = tmpDirs.createNew("writeFailFailsFlush", "logs"); + NativeIO io = new NativeIOImpl() { + boolean failed = false; + @Override + public int pwrite(int fd, long pointer, int count, long offset) throws NativeIOException { + synchronized (this) { + if (!failed) { + failed = true; + throw new NativeIOException("fail for test"); + } + } + return super.pwrite(fd, pointer, count, offset); + } + }; + + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, io, Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, 0xdededede); + writer.writeAt(0, bb); + writer.flush(); + } + } + + @Test + public void testWriteWithPadding() throws Exception { + File ledgerDir = tmpDirs.createNew("paddingWrite", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, 0xdededede); + bb.writerIndex(123); + writer.writeDelimited(bb); + writer.flush(); + } + + ByteBuf contents = readIntoByteBuf(ledgerDir, 5678); + assertThat(contents.readInt(), equalTo(123)); + for (int i = 0; i < 123; i++) { + assertThat(contents.readByte(), equalTo((byte) 0xde)); + } + for (int i = 0; i < Buffer.ALIGNMENT - (123 + Integer.BYTES); i++) { + assertThat(contents.readByte(), equalTo(Buffer.PADDING_BYTE)); + } + while (contents.isReadable()) { + assertThat((int) contents.readByte(), equalTo(0)); + } + } + + @Test + public void testWriteBlocksFlush() throws Exception { + ExecutorService flushExecutor = Executors.newSingleThreadExecutor(); + try { + File ledgerDir = tmpDirs.createNew("blockWrite", "logs"); + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), + 1 << 24, writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + CompletableFuture blocker = new CompletableFuture<>(); + writeExecutor.submit(() -> { + blocker.join(); + return null; + }); + ByteBuf bb = Unpooled.buffer(4096); + TestBuffer.fillByteBuf(bb, 0xdeadbeef); + writer.writeAt(0, bb); + Future f = flushExecutor.submit(() -> { + writer.flush(); + return null; + }); + Thread.sleep(100); + assertThat(f.isDone(), equalTo(false)); + blocker.complete(null); + f.get(); + } + ByteBuf contents = readIntoByteBuf(ledgerDir, 1234); + for (int i = 0; i < 4096 / Integer.BYTES; i++) { + assertThat(contents.readInt(), equalTo(0xdeadbeef)); + } + if (contents.readableBytes() > 0) { // linux-only: fallocate will preallocate file + while (contents.isReadable()) { + assertThat((int) contents.readByte(), equalTo(0)); + } + } + } finally { + flushExecutor.shutdownNow(); + } + } + + @Test(expected = IOException.class) + public void testFailsToOpen() throws Exception { + File ledgerDir = tmpDirs.createNew("failOpen", "logs"); + ledgerDir.delete(); + + BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + try { + new DirectWriter(1234, logFilename(ledgerDir, 1234), + 1 << 30, MoreExecutors.newDirectExecutorService(), + buffers, new NativeIOImpl(), Slogger.CONSOLE); + } finally { + buffers.close(); + } + } + + @Test + public void fallocateNotAvailable() throws Exception { + File ledgerDir = tmpDirs.createNew("fallocUnavailable", "logs"); + NativeIO nativeIO = new NativeIOImpl() { + @Override + public int fallocate(int fd, int mode, long offset, long len) + throws NativeIOException { + throw new NativeIOException("pretending I'm a mac"); + } + }; + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + LogWriter writer = new DirectWriter(3456, logFilename(ledgerDir, 3456), + 1 << 24, writeExecutor, + buffers, nativeIO, Slogger.CONSOLE)) { + ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); + TestBuffer.fillByteBuf(bb, 0xdeadbeef); + + writer.writeAt(0, bb); + writer.flush(); + } + + // should be 0xdeadbeef until the end of the file + ByteBuf contents = readIntoByteBuf(ledgerDir, 3456); + assertThat(contents.readableBytes(), equalTo(Buffer.ALIGNMENT)); + while (contents.isReadable()) { + assertThat(contents.readInt(), equalTo(0xdeadbeef)); + } + } + + @Test + public void testWriteAtIntLimit() throws Exception { + File ledgerDir = tmpDirs.createNew("intLimit", "logs"); + + try (BufferPool buffers = new BufferPool(new NativeIOImpl(), 1 << 14, 8); + LogWriter writer = new DirectWriter(3456, logFilename(ledgerDir, 3456), + (long) Integer.MAX_VALUE + (Buffer.ALIGNMENT * 100), + writeExecutor, + buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + ByteBuf b1 = Unpooled.buffer(Buffer.ALIGNMENT - (Integer.BYTES * 2) - 1); + TestBuffer.fillByteBuf(b1, 0xdeadbeef); + + long finalSeekablePosition = Integer.MAX_VALUE & ~(Buffer.ALIGNMENT - 1); + writer.position(finalSeekablePosition); + long offset = writer.writeDelimited(b1); + assertThat(offset, equalTo(finalSeekablePosition + Integer.BYTES)); + assertThat(writer.position(), equalTo((long) Integer.MAX_VALUE - Integer.BYTES)); + + offset = writer.writeDelimited(b1); + assertThat(offset, equalTo((long) Integer.MAX_VALUE)); + + writer.flush(); + + try { + writer.writeDelimited(b1); + Assert.fail("Shouldn't be possible, we've gone past MAX_INT"); + } catch (IOException ioe) { + // expected + } + } + + } + + static ByteBuf readIntoByteBuf(File directory, int logId) throws Exception { + byte[] bytes = new byte[1024]; + File file = new File(DirectEntryLogger.logFilename(directory, logId)); + slog.kv("filename", file.toString()).info("reading in"); + ByteBuf byteBuf = Unpooled.buffer((int) file.length()); + try (FileInputStream is = new FileInputStream(file)) { + int bytesRead = is.read(bytes); + while (bytesRead > 0) { + byteBuf.writeBytes(bytes, 0, bytesRead); + bytesRead = is.read(bytes); + } + } + + assertThat(byteBuf.readableBytes(), equalTo((int) file.length())); + return byteBuf; + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestTransactionalEntryLogCompactor.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestTransactionalEntryLogCompactor.java new file mode 100644 index 00000000000..bf37ebf5af2 --- /dev/null +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestTransactionalEntryLogCompactor.java @@ -0,0 +1,615 @@ +/** + * + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + * + */ +package org.apache.bookkeeper.bookie.storage.directentrylogger; + +import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTED_SUFFIX; +import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; + +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.assertEntryEquals; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.logIdFromLocation; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirectEntryLogger; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirsManager; +import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newLegacyEntryLogger; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; + +import com.google.common.util.concurrent.MoreExecutors; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufAllocator; + +import java.io.File; +import java.io.IOException; + +import java.util.Arrays; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.bookkeeper.bookie.EntryLocation; +import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.MockLedgerStorage; +import org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor; +import org.apache.bookkeeper.bookie.storage.CompactionEntryLog; +import org.apache.bookkeeper.bookie.storage.EntryLogIdsImpl; +import org.apache.bookkeeper.bookie.storage.EntryLogScanner; +import org.apache.bookkeeper.bookie.storage.EntryLogger; +import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.conf.ServerConfiguration; +import org.apache.bookkeeper.slogger.Slogger; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.bookkeeper.test.TmpDirs; +import org.junit.After; +import org.junit.Test; + +/** + * TestTransactionalEntryLogCompactor. + */ +public class TestTransactionalEntryLogCompactor { + private static final Slogger slog = Slogger.CONSOLE; + + private final TmpDirs tmpDirs = new TmpDirs(); + private static final long deadLedger = 1L; + private static final long liveLedger = 2L; + + @After + public void cleanup() throws Exception { + tmpDirs.cleanup(); + } + + @Test + public void testHappyCase() throws Exception { + File ledgerDir = tmpDirs.createNew("compactHappyCase", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(true)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1)); + EntryLocation loc = ledgerStorage.getUpdatedLocations().get(0); + + long compactedLogId = logIdFromLocation(loc.getLocation()); + assertThat(compactedLogId, not(equalTo(logId))); + assertThat(loc.getLedger(), equalTo(liveLedger)); + assertThat(loc.getEntry(), equalTo(2L)); + + meta = entryLogger.getEntryLogMetadata(compactedLogId); + assertThat(meta.containsLedger(deadLedger), equalTo(false)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + Integer.BYTES)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + ByteBuf bb = entryLogger.readEntry(loc.getLedger(), loc.getEntry(), loc.getLocation()); + assertEntryEquals(bb, makeEntry(liveLedger, 2L, 1000, (byte) 0xfa)); + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + } + } + + @Test + public void testHappyCase1000() throws Exception { + File ledgerDir = tmpDirs.createNew("compactHappyCase1000", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData1000(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo((1000L + Integer.BYTES) * 1000 * 2)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(true)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1000)); + long compactedLogId = -1; + for (int i = 0; i < 1000; i++) { + EntryLocation loc = ledgerStorage.getUpdatedLocations().get(i); + compactedLogId = logIdFromLocation(loc.getLocation()); + assertThat(compactedLogId, not(equalTo(logId))); + assertThat(loc.getLedger(), equalTo(liveLedger)); + assertThat(loc.getEntry(), equalTo(Long.valueOf(i))); + + ByteBuf bb = entryLogger.readEntry(loc.getLedger(), loc.getEntry(), loc.getLocation()); + assertEntryEquals(bb, makeEntry(liveLedger, i, 1000, (byte) (0xfa + i))); + } + + meta = entryLogger.getEntryLogMetadata(compactedLogId); + assertThat(meta.containsLedger(deadLedger), equalTo(false)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo((1000L + Integer.BYTES) * 1000)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + } + } + + @Test + public void testScanFail() throws Exception { + File ledgerDir = tmpDirs.createNew("compactScanFail", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLoggerFailAdd(ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(false)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(0)); + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), empty()); + } + } + + @Test + public void testScanFailNoAbortAndContinue() throws Exception { + File ledgerDir = tmpDirs.createNew("compactScanFail", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLoggerFailAddNoAbort(ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(false)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(0)); + assertThat(compactingFiles(curDir).size(), equalTo(1)); + assertThat(compactedFiles(curDir), empty()); + } + + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + compactor.cleanUpAndRecover(); + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), empty()); + + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(true)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1)); + + EntryLocation loc = ledgerStorage.getUpdatedLocations().get(0); + + long compactedLogId = logIdFromLocation(loc.getLocation()); + assertThat(compactedLogId, not(equalTo(logId))); + assertThat(loc.getLedger(), equalTo(liveLedger)); + assertThat(loc.getEntry(), equalTo(2L)); + } + } + + @Test + public void testFlushFail() throws Exception { + File ledgerDir = tmpDirs.createNew("compactScanFail", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLoggerFailFlush(ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(false)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(0)); + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), empty()); + } + } + + @Test + public void testMarkCompactFailNoAbort() throws Exception { + File ledgerDir = tmpDirs.createNew("compactScanFail", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLoggerFailMarkCompactedNoAbort(ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(false)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(0)); + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), hasSize(1)); + } + + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + assertThat(entryLogger.logExists(logId), equalTo(true)); + CompletableFuture removedId = new CompletableFuture<>(); + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> removedId.complete(removedLogId)); + compactor.cleanUpAndRecover(); + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), empty()); + + assertThat(removedId.isDone(), equalTo(true)); + assertThat(removedId.get(), equalTo(logId)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1)); + + EntryLocation loc = ledgerStorage.getUpdatedLocations().get(0); + + long compactedLogId = logIdFromLocation(loc.getLocation()); + assertThat(compactedLogId, not(equalTo(logId))); + assertThat(loc.getLedger(), equalTo(liveLedger)); + assertThat(loc.getEntry(), equalTo(2L)); + + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(compactedLogId); + assertThat(meta.containsLedger(deadLedger), equalTo(false)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + Integer.BYTES)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + ByteBuf bb = entryLogger.readEntry(loc.getLedger(), loc.getEntry(), loc.getLocation()); + assertEntryEquals(bb, makeEntry(liveLedger, 2L, 1000, (byte) 0xfa)); + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + } + } + + @Test + public void testIndexFail() throws Exception { + File ledgerDir = tmpDirs.createNew("compactScanFail", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData(ledgerDir); + MockLedgerStorage ledgerStorageFailFlush = new MockLedgerStorage() { + @Override + public void flushEntriesLocationsIndex() throws IOException { + throw new IOException("fail on flush"); + } + }; + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorageFailFlush, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + assertThat(meta.containsLedger(deadLedger), equalTo(true)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + 1000 + (Integer.BYTES * 2))); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(false)); + + assertThat(ledgerStorageFailFlush.getUpdatedLocations(), hasSize(1)); + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), hasSize(1)); + } + + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + CompletableFuture removedId = new CompletableFuture<>(); + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> removedId.complete(removedLogId)); + assertThat(entryLogger.logExists(logId), equalTo(true)); + compactor.cleanUpAndRecover(); + assertThat(compactingFiles(curDir), empty()); + assertThat(compactedFiles(curDir), empty()); + + assertThat(removedId.isDone(), equalTo(true)); + assertThat(removedId.get(), equalTo(logId)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1)); + + EntryLocation loc = ledgerStorage.getUpdatedLocations().get(0); + + long compactedLogId = logIdFromLocation(loc.getLocation()); + assertThat(compactedLogId, not(equalTo(logId))); + assertThat(loc.getLedger(), equalTo(liveLedger)); + assertThat(loc.getEntry(), equalTo(2L)); + + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(compactedLogId); + assertThat(meta.containsLedger(deadLedger), equalTo(false)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo(1000L + Integer.BYTES)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + + ByteBuf bb = entryLogger.readEntry(loc.getLedger(), loc.getEntry(), loc.getLocation()); + assertEntryEquals(bb, makeEntry(liveLedger, 2L, 1000, (byte) 0xfa)); + assertThat(entryLogger.incompleteCompactionLogs(), empty()); + } + } + + @Test + public void testMetadataWritten() throws Exception { + File ledgerDir = tmpDirs.createNew("compactHappyCase", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + long logId = writeLogData1000(ledgerDir); + MockLedgerStorage ledgerStorage = new MockLedgerStorage(); + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + TransactionalEntryLogCompactor compactor = new TransactionalEntryLogCompactor( + new ServerConfiguration(), + entryLogger, + ledgerStorage, + (removedLogId) -> {}); + EntryLogMetadata meta = entryLogger.getEntryLogMetadata(logId); + meta.removeLedgerIf((ledgerId) -> ledgerId == deadLedger); + assertThat(compactor.compact(meta), equalTo(true)); + + assertThat(ledgerStorage.getUpdatedLocations(), hasSize(1000)); + long compactedLogId = logIdFromLocation( + ledgerStorage.getUpdatedLocations().get(0).getLocation()); + + meta = ((DirectEntryLogger) entryLogger).readEntryLogIndex(compactedLogId); + assertThat(meta.containsLedger(deadLedger), equalTo(false)); + assertThat(meta.containsLedger(liveLedger), equalTo(true)); + assertThat(meta.getTotalSize(), equalTo((1000L + Integer.BYTES) * 1000)); + assertThat(meta.getRemainingSize(), equalTo(meta.getTotalSize())); + } + } + + Set compactingFiles(File dir) throws Exception { + return Arrays.stream(dir.listFiles((f) -> f.getName().endsWith(COMPACTING_SUFFIX))) + .collect(Collectors.toSet()); + } + + Set compactedFiles(File dir) throws Exception { + return Arrays.stream(dir.listFiles((f) -> f.getName().endsWith(COMPACTED_SUFFIX))) + .collect(Collectors.toSet()); + } + + int writeLogData(File ledgerDir) throws Exception { + try (EntryLogger entryLogger = newLegacyEntryLogger(2 << 20, ledgerDir)) { + long loc1 = entryLogger.addEntry(deadLedger, makeEntry(deadLedger, 1L, 1000, (byte) 0xde)); + long loc2 = entryLogger.addEntry(liveLedger, makeEntry(liveLedger, 2L, 1000, (byte) 0xfa)); + assertThat(logIdFromLocation(loc1), equalTo(logIdFromLocation(loc2))); + return logIdFromLocation(loc2); + } + } + + int writeLogData1000(File ledgerDir) throws Exception { + try (EntryLogger entryLogger = newDirectEntryLogger(2 << 20, ledgerDir)) { + long loc1, loc2 = -1; + for (int i = 0; i < 1000; i++) { + loc1 = entryLogger.addEntry(deadLedger, makeEntry(deadLedger, i, 1000, (byte) (0xde + i))); + if (loc2 != -1) { + assertThat(logIdFromLocation(loc1), equalTo(logIdFromLocation(loc2))); + } + loc2 = entryLogger.addEntry(liveLedger, makeEntry(liveLedger, i, 1000, (byte) (0xfa + i))); + assertThat(logIdFromLocation(loc1), equalTo(logIdFromLocation(loc2))); + } + return logIdFromLocation(loc2); + } + } + + private static DirectEntryLogger newDirectEntryLoggerFailAdd(File ledgerDir) throws Exception { + return newDirectEntryLoggerCompactionOverride( + ledgerDir, + (cel) -> new CompactionEntryLogProxy(cel) { + @Override + public long addEntry(long ledgerId, ByteBuf entry) throws IOException { + throw new IOException("Don't allow adds"); + } + }); + } + + private static DirectEntryLogger newDirectEntryLoggerFailAddNoAbort(File ledgerDir) throws Exception { + return newDirectEntryLoggerCompactionOverride( + ledgerDir, + (cel) -> new CompactionEntryLogProxy(cel) { + @Override + public long addEntry(long ledgerId, ByteBuf entry) throws IOException { + throw new IOException("Don't allow adds"); + } + + @Override + public void abort() {} + }); + } + + private static DirectEntryLogger newDirectEntryLoggerFailFlush(File ledgerDir) throws Exception { + return newDirectEntryLoggerCompactionOverride( + ledgerDir, + (cel) -> new CompactionEntryLogProxy(cel) { + @Override + public void flush() throws IOException { + throw new IOException("No flushing"); + } + }); + } + + private static DirectEntryLogger newDirectEntryLoggerFailMarkCompactedNoAbort(File ledgerDir) throws Exception { + return newDirectEntryLoggerCompactionOverride( + ledgerDir, + (cel) -> new CompactionEntryLogProxy(cel) { + @Override + public void markCompacted() throws IOException { + super.markCompacted(); + throw new IOException("No compact"); + } + + @Override + public void abort() {} + }); + } + + private static DirectEntryLogger newDirectEntryLoggerCompactionOverride( + File ledgerDir, + Function override) throws Exception { + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + + return new DirectEntryLogger( + curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 2 << 20, // max file size + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 4 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE) { + @Override + public CompactionEntryLog newCompactionLog(long logToCompact) throws IOException { + return override.apply(super.newCompactionLog(logToCompact)); + } + }; + } + + private static class CompactionEntryLogProxy implements CompactionEntryLog { + protected final CompactionEntryLog delegate; + + CompactionEntryLogProxy(CompactionEntryLog delegate) { + this.delegate = delegate; + } + + @Override + public long addEntry(long ledgerId, ByteBuf entry) throws IOException { + return delegate.addEntry(ledgerId, entry); + } + + @Override + public void scan(EntryLogScanner scanner) throws IOException { + delegate.scan(scanner); + } + + @Override + public void flush() throws IOException { + delegate.flush(); + } + + @Override + public void abort() { + delegate.abort(); + } + + @Override + public void markCompacted() throws IOException { + delegate.markCompacted(); + } + + @Override + public void makeAvailable() throws IOException { + delegate.makeAvailable(); + } + + @Override + public void finalizeAndCleanup() { + delegate.finalizeAndCleanup(); + } + + @Override + public long getDstLogId() { + return delegate.getDstLogId(); + } + + @Override + public long getSrcLogId() { + return delegate.getSrcLogId(); + } + } +} diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java index 955115351f1..c6cdfcb16f1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java @@ -33,6 +33,7 @@ import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.TestBookieImpl; +import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; @@ -54,20 +55,23 @@ private static class MockedDbLedgerStorage extends DbLedgerStorage { @Override protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, + EntryLogger entryLogger, StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { return new MockedSingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, + entryLogger, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } private static class MockedSingleDirectoryDbLedgerStorage extends SingleDirectoryDbLedgerStorage { public MockedSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, - LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger, + LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, + EntryLogger entrylogger, StatsLogger statsLogger, ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { - super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, + super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, entrylogger, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } From 73aae68a4e7bbf19a4e91f44f8af275258359a23 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sat, 7 May 2022 15:20:59 +0800 Subject: [PATCH 02/15] format code --- .../org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java | 1 - .../directentrylogger/{BufferTest.java => TestBuffer.java} | 2 +- .../bookie/storage/directentrylogger/TestDirectReader.java | 3 ++- .../bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) rename bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/{BufferTest.java => TestBuffer.java} (99%) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java index cf73de46229..b0a85551f7c 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java @@ -35,7 +35,6 @@ import io.netty.buffer.ByteBuf; import java.io.File; -import org.apache.bookkeeper.bookie.EntryLogger; import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectCompactionEntryLog; import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; import org.apache.bookkeeper.slogger.Slogger; diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestBuffer.java similarity index 99% rename from bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferTest.java rename to bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestBuffer.java index 2a48157523f..40f2aaac9fd 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestBuffer.java @@ -38,7 +38,7 @@ /** * TestBuffer. */ -public class BufferTest { +public class TestBuffer { @Test public void testIsAligned() throws Exception { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java index e7d810f99b0..126cdbce58f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java @@ -280,7 +280,8 @@ buffers, new NativeIOImpl(), Slogger.CONSOLE)) { ByteBufAllocator.DEFAULT, new NativeIOImpl(), Buffer.ALIGNMENT, 1 << 20, opLogger)) { - List> offset2PatternList = new ArrayList(offset2Pattern.entrySet()); + List> offset2PatternList = + new ArrayList>(offset2Pattern.entrySet()); Collections.shuffle(offset2PatternList); for (Map.Entry e : offset2PatternList) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java index c6cdfcb16f1..22281e2d5e4 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java @@ -72,7 +72,7 @@ public MockedSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerMana ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, entrylogger, - statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); + statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } @Override From 026649442f43e3c47aab90172564ce70bd27802b Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 10:01:38 +0800 Subject: [PATCH 03/15] format code --- .../directentrylogger/DirectEntryLogger.java | 3 +- .../storage/directentrylogger/Events.java | 2 +- .../storage/directentrylogger/LogWriter.java | 2 +- .../bookie/storage/ldb/DbLedgerStorage.java | 110 ++---------------- .../ldb/SingleDirectoryDbLedgerStorage.java | 8 +- .../ldb/DbLedgerStorageWriteCacheTest.java | 10 +- 6 files changed, 16 insertions(+), 119 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java index 467864d814e..cdf65cf62fc 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java @@ -65,7 +65,6 @@ * DirectEntryLogger. */ public class DirectEntryLogger implements EntryLogger { - private static final String LOGFILE_SUFFIX = ".log"; private final Slogger slog; private final File ledgerDir; private final EntryLogIds ids; @@ -462,7 +461,7 @@ private LogWriter newDirectWriter(int newId) throws IOException { } public static File logFile(File directory, int logId) { - return new File(directory, Long.toHexString(logId) + LOGFILE_SUFFIX); + return new File(directory, Long.toHexString(logId) + LOG_FILE_SUFFIX); } public static String logFilename(File directory, int logId) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java index 5fdc96ff3ce..b9ca1706e4f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java @@ -55,7 +55,7 @@ public enum Events { ENTRYLOGGER_MISCONFIGURED, /** - * The entrylogger has started writing the a new log file. The previous log file may not + * The entrylogger has started writing a new log file. The previous log file may not * be entirely flushed when this is called, though they will be after an explicit flush call. */ LOG_ROLL, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java index 6135b146ef3..19ba003423a 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogWriter.java @@ -39,7 +39,7 @@ interface LogWriter extends AutoCloseable { void writeAt(long offset, ByteBuf buf) throws IOException; /** - * Write a delimited buffer the the log. The size of the buffer is first + * Write a delimited buffer the log. The size of the buffer is first * written and then the buffer itself. * Note that the returned offset is for the buffer itself, not the size. * So, if a buffer is written at the start of the file, the returned offset diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 425f0ef2c75..7d33e5eb5e9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java @@ -39,7 +39,6 @@ import java.util.EnumSet; import java.util.List; import java.util.PrimitiveIterator.OfLong; -import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -50,24 +49,18 @@ import org.apache.bookkeeper.bookie.CheckpointSource; import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint; import org.apache.bookkeeper.bookie.Checkpointer; -import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.GarbageCollectionStatus; import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification; import org.apache.bookkeeper.bookie.LedgerCache; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.LedgerStorage; import org.apache.bookkeeper.bookie.StateManager; -import org.apache.bookkeeper.bookie.storage.EntryLogIdsImpl; -import org.apache.bookkeeper.bookie.storage.EntryLogger; -import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType; import org.apache.bookkeeper.bookie.storage.ldb.SingleDirectoryDbLedgerStorage.LedgerLoggerProcessor; import org.apache.bookkeeper.common.util.MathUtils; import org.apache.bookkeeper.common.util.Watcher; -import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.meta.LedgerManager; -import org.apache.bookkeeper.slogger.slf4j.Slf4jSlogger; import org.apache.bookkeeper.stats.Gauge; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.stats.StatsLogger; @@ -75,6 +68,7 @@ import org.apache.bookkeeper.util.DiskChecker; import org.apache.commons.lang3.StringUtils; + /** * Implementation of LedgerStorage that uses RocksDB to keep the indexes for entries stored in EntryLogs. */ @@ -84,18 +78,6 @@ public class DbLedgerStorage implements LedgerStorage { public static final String WRITE_CACHE_MAX_SIZE_MB = "dbStorage_writeCacheMaxSizeMb"; public static final String READ_AHEAD_CACHE_MAX_SIZE_MB = "dbStorage_readAheadCacheMaxSizeMb"; - static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize"; - public static final String DIRECT_IO_ENTRYLOGGER = "dbStorage_directIOEntryLogger"; - public static final String DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB = - "dbStorage_directIOEntryLoggerTotalWriteBufferSizeMb"; - public static final String DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB = - "dbStorage_directIOEntryLoggerTotalReadBufferSizeMb"; - public static final String DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB = - "dbStorage_directIOEntryLoggerReadBufferSizeMb"; - public static final String DIRECT_IO_ENTRYLOGGER_MAX_READ_FDS_PER_THREAD = - "dbStorage_directIOEntryLoggerMaxReadFdsPerThread"; - public static final String DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS = - "dbStorage_directIOEntryLoggerMaxFdCacheTimeSeconds"; static final String MAX_THROTTLE_TIME_MILLIS = "dbStorage_maxThrottleTimeMs"; @@ -106,18 +88,9 @@ public class DbLedgerStorage implements LedgerStorage { private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory()) / MB; + static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize"; private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100; - private static final long DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB = - (long) (0.125 * PlatformDependent.maxDirectMemory()) - / MB; - private static final long DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB = - (long) (0.125 * PlatformDependent.maxDirectMemory()) - / MB; - private static final long DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB = 8; - - private static final int DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS = 300; - // use the storage assigned to ledger 0 for flags. // if the storage configuration changes, the flags may be lost // but in that case data integrity should kick off anyhow. @@ -127,9 +100,6 @@ public class DbLedgerStorage implements LedgerStorage { // Keep 1 single Bookie GC thread so the the compactions from multiple individual directories are serialized private ScheduledExecutorService gcExecutor; - private ExecutorService entryLoggerWriteExecutor = null; - private ExecutorService entryLoggerFlushExecutor = null; - private DbLedgerStorageStats stats; protected ByteBufAllocator allocator; @@ -157,7 +127,7 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le DEFAULT_WRITE_CACHE_MAX_SIZE_MB) * MB; long readCacheMaxSize = getLongVariableOrDefault(conf, READ_AHEAD_CACHE_MAX_SIZE_MB, DEFAULT_READ_CACHE_MAX_SIZE_MB) * MB; - boolean directIOEntryLogger = getBooleanVariableOrDefault(conf, DIRECT_IO_ENTRYLOGGER, false); + this.allocator = allocator; this.numberOfDirs = ledgerDirsManager.getAllLedgerDirs().size(); @@ -183,55 +153,9 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le // Remove the `/current` suffix which will be appended again by LedgersDirManager dirs[0] = ledgerDir.getParentFile(); LedgerDirsManager ldm = new LedgerDirsManager(conf, dirs, ledgerDirsManager.getDiskChecker(), statsLogger); - - EntryLogger entrylogger; - if (directIOEntryLogger) { - long perDirectoryTotalWriteBufferSize = MB * getLongVariableOrDefault( - conf, - DIRECT_IO_ENTRYLOGGER_TOTAL_WRITEBUFFER_SIZE_MB, - DEFAULT_DIRECT_IO_TOTAL_WRITEBUFFER_SIZE_MB) / numberOfDirs; - long perDirectoryTotalReadBufferSize = MB * getLongVariableOrDefault( - conf, - DIRECT_IO_ENTRYLOGGER_TOTAL_READBUFFER_SIZE_MB, - DEFAULT_DIRECT_IO_TOTAL_READBUFFER_SIZE_MB) / numberOfDirs; - int readBufferSize = MB * (int) getLongVariableOrDefault( - conf, - DIRECT_IO_ENTRYLOGGER_READBUFFER_SIZE_MB, - DEFAULT_DIRECT_IO_READBUFFER_SIZE_MB); - int maxFdCacheTimeSeconds = (int) getLongVariableOrDefault( - conf, - DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS, - DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS); - Slf4jSlogger slog = new Slf4jSlogger(DbLedgerStorage.class); - entryLoggerWriteExecutor = Executors.newSingleThreadExecutor( - new DefaultThreadFactory("EntryLoggerWrite")); - entryLoggerFlushExecutor = Executors.newSingleThreadExecutor( - new DefaultThreadFactory("EntryLoggerFlush")); - - int numReadThreads = conf.getNumReadWorkerThreads(); - if (numReadThreads == 0) { - numReadThreads = conf.getServerNumIOThreads(); - } - - entrylogger = new DirectEntryLogger(ledgerDir, new EntryLogIdsImpl(ledgerDirsManager, slog), - new NativeIOImpl(), - allocator, entryLoggerWriteExecutor, entryLoggerFlushExecutor, - conf.getEntryLogSizeLimit(), - conf.getNettyMaxFrameSizeBytes() - 500, - perDirectoryTotalWriteBufferSize, - perDirectoryTotalReadBufferSize, - readBufferSize, - numReadThreads, - maxFdCacheTimeSeconds, - slog, statsLogger); - } else { - entrylogger = new DefaultEntryLogger(conf, ldm, null, statsLogger, allocator); - } - ledgerStorageList.add(newSingleDirectoryDbLedgerStorage(conf, ledgerManager, ldm, - indexDirsManager, entrylogger, - statsLogger, gcExecutor, perDirectoryWriteCacheSize, - perDirectoryReadCacheSize, - readAheadCacheBatchSize)); + ledgerStorageList.add(newSingleDirectoryDbLedgerStorage(conf, ledgerManager, ldm, indexDirsManager, + statsLogger, gcExecutor, perDirectoryWriteCacheSize, + perDirectoryReadCacheSize, readAheadCacheBatchSize)); ldm.getListeners().forEach(ledgerDirsManager::addLedgerDirsListener); } @@ -266,11 +190,10 @@ public Long getSample() { @VisibleForTesting protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, - EntryLogger entryLogger, StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { - return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, entryLogger, + return new SingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } @@ -298,12 +221,6 @@ public void shutdown() throws InterruptedException { for (LedgerStorage ls : ledgerStorageList) { ls.shutdown(); } - if (entryLoggerWriteExecutor != null) { - entryLoggerWriteExecutor.shutdown(); - } - if (entryLoggerFlushExecutor != null) { - entryLoggerFlushExecutor.shutdown(); - } } @Override @@ -515,19 +432,6 @@ static long getLongVariableOrDefault(ServerConfiguration conf, String keyName, l } } - static boolean getBooleanVariableOrDefault(ServerConfiguration conf, String keyName, boolean defaultValue) { - Object obj = conf.getProperty(keyName); - if (obj instanceof Boolean) { - return (Boolean) obj; - } else if (obj == null) { - return defaultValue; - } else if (StringUtils.isEmpty(conf.getString(keyName))) { - return defaultValue; - } else { - return conf.getBoolean(keyName); - } - } - @Override public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException { // check Issue #2078 diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java index 4e72a2ebddd..30516f82c67 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java @@ -146,8 +146,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage private final Counter flushExecutorTime; public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, - LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, - EntryLogger entryLogger, StatsLogger statsLogger, + LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { checkArgument(ledgerDirsManager.getAllLedgerDirs().size() == 1, @@ -187,9 +186,8 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TimeUnit.MINUTES); - this.entryLogger = entryLogger; - gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, entryLogger, - statsLogger); + entryLogger = new DefaultEntryLogger(conf, ledgerDirsManager, null, statsLogger, allocator); + gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, entryLogger, statsLogger); dbLedgerStorageStats = new DbLedgerStorageStats( ledgerDirStatsLogger, diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java index 22281e2d5e4..955115351f1 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageWriteCacheTest.java @@ -33,7 +33,6 @@ import org.apache.bookkeeper.bookie.BookieImpl; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.TestBookieImpl; -import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.conf.TestBKConfiguration; import org.apache.bookkeeper.meta.LedgerManager; @@ -55,24 +54,21 @@ private static class MockedDbLedgerStorage extends DbLedgerStorage { @Override protected SingleDirectoryDbLedgerStorage newSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, - EntryLogger entryLogger, StatsLogger statsLogger, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { return new MockedSingleDirectoryDbLedgerStorage(conf, ledgerManager, ledgerDirsManager, indexDirsManager, - entryLogger, statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } private static class MockedSingleDirectoryDbLedgerStorage extends SingleDirectoryDbLedgerStorage { public MockedSingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager ledgerManager, - LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, - EntryLogger entrylogger, StatsLogger statsLogger, + LedgerDirsManager ledgerDirsManager, LedgerDirsManager indexDirsManager, StatsLogger statsLogger, ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize, int readAheadCacheBatchSize) throws IOException { - super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, entrylogger, - statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); + super(conf, ledgerManager, ledgerDirsManager, indexDirsManager, + statsLogger, allocator, gcExecutor, writeCacheSize, readCacheSize, readAheadCacheBatchSize); } @Override From 284d502018e360a79c3ad06466add41c4a7c1fff Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 15:45:15 +0800 Subject: [PATCH 04/15] fix spotbugs check --- .../bookie/storage/EntryLogIdsImpl.java | 33 ++++++--- .../DirectCompactionEntryLog.java | 14 ++-- .../directentrylogger/DirectEntryLogger.java | 73 ++++++++++--------- .../directentrylogger/DirectReader.java | 1 - .../directentrylogger/DirectWriter.java | 2 +- .../directentrylogger/LogMetadata.java | 1 - native-io/pom.xml | 1 + .../common/util/nativeio/NativeIOJni.java | 2 +- 8 files changed, 73 insertions(+), 54 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java index f2b070088a5..f8df75c7c48 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java @@ -122,12 +122,16 @@ static int[] findLargestGap(List currentIds) { public static List logIdsInDirectory(File directory) { List ids = new ArrayList<>(); - File[] files = directory.listFiles(); - for (File f : files) { - Matcher m = FILE_PATTERN.matcher(f.getName()); - if (m.matches()) { - int logId = Integer.parseUnsignedInt(m.group(1), 16); - ids.add(logId); + if (directory.exists() && directory.isDirectory()) { + File[] files = directory.listFiles(); + if (files != null && files.length > 0) { + for (File f : files) { + Matcher m = FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int logId = Integer.parseUnsignedInt(m.group(1), 16); + ids.add(logId); + } + } } } return ids; @@ -135,14 +139,19 @@ public static List logIdsInDirectory(File directory) { private static List compactedLogIdsInDirectory(File directory) { List ids = new ArrayList<>(); - File[] files = directory.listFiles(); - for (File f : files) { - Matcher m = COMPACTED_FILE_PATTERN.matcher(f.getName()); - if (m.matches()) { - int logId = Integer.parseUnsignedInt(m.group(1), 16); - ids.add(logId); + if (directory.exists() && directory.isDirectory()) { + File[] files = directory.listFiles(); + if (files != null && files.length > 0) { + for (File f : files) { + Matcher m = COMPACTED_FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int logId = Integer.parseUnsignedInt(m.group(1), 16); + ids.add(logId); + } + } } } return ids; + } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java index cce4cc63e99..6e7d0f98ee7 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java @@ -252,11 +252,15 @@ public void markCompacted() throws IOException { writer.finalizeAndClose(); idempotentLink(compactingFile, compactedFile); - compactingFile.delete(); - - slog.kv("compactingFile", compactingFile) - .kv("compactedFile", compactedFile) - .info(Events.COMPACTION_MARK_COMPACTED); + if (!compactingFile.delete()) { + slog.kv("compactingFile", compactingFile) + .kv("compactedFile", compactedFile) + .info(Events.COMPACTION_DELETE_FAILURE); + } else { + slog.kv("compactingFile", compactingFile) + .kv("compactedFile", compactedFile) + .info(Events.COMPACTION_MARK_COMPACTED); + } } @Override diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java index cdf65cf62fc..362d074d9bd 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java @@ -300,7 +300,10 @@ public void flush() throws IOException { } private Future flushCurrent() throws IOException { - WriterWithMetadata flushWriter = this.curWriter; + WriterWithMetadata flushWriter; + synchronized (this) { + flushWriter = this.curWriter; + } if (flushWriter != null) { return flushExecutor.submit(() -> { long start = System.nanoTime(); @@ -331,21 +334,20 @@ private void flushAndCloseCurrent() throws IOException { pendingFlushes.add(flushPromise); } if (flushWriter != null) { - flushExecutor.submit(() -> { - long start = System.nanoTime(); - try { - flushWriter.finalizeAndClose(); - stats.getWriterFlushStats() - .registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); - unflushedLogs.remove(flushWriter.logId()); - flushPromise.complete(null); - } catch (Throwable t) { - stats.getWriterFlushStats() - .registerFailedEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); - flushPromise.completeExceptionally(t); - } - return null; - }); + flushExecutor.execute(() -> { + long start = System.nanoTime(); + try { + flushWriter.finalizeAndClose(); + stats.getWriterFlushStats() + .registerSuccessfulEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + unflushedLogs.remove(flushWriter.logId()); + flushPromise.complete(null); + } catch (Throwable t) { + stats.getWriterFlushStats() + .registerFailedEvent(System.nanoTime() - start, TimeUnit.NANOSECONDS); + flushPromise.completeExceptionally(t); + } + }); } else { flushPromise.complete(null); } @@ -480,25 +482,30 @@ public CompactionEntryLog newCompactionLog(long srcLogId) throws IOException { public Collection incompleteCompactionLogs() { List logs = new ArrayList<>(); - for (File f : ledgerDir.listFiles()) { - if (f.getName().endsWith(COMPACTING_SUFFIX)) { - try { - Files.deleteIfExists(f.toPath()); - } catch (IOException ioe) { - slog.kv("file", f).warn(Events.COMPACTION_DELETE_FAILURE); - } - } + if (ledgerDir.exists() && ledgerDir.isDirectory()) { + File[] files = ledgerDir.listFiles(); + if (files != null && files.length > 0) { + for (File f : files) { + if (f.getName().endsWith(COMPACTING_SUFFIX)) { + try { + Files.deleteIfExists(f.toPath()); + } catch (IOException ioe) { + slog.kv("file", f).warn(Events.COMPACTION_DELETE_FAILURE); + } + } - Matcher m = EntryLogIdsImpl.COMPACTED_FILE_PATTERN.matcher(f.getName()); - if (m.matches()) { - int dstLogId = Integer.parseUnsignedInt(m.group(1), 16); - int srcLogId = Integer.parseUnsignedInt(m.group(2), 16); + Matcher m = EntryLogIdsImpl.COMPACTED_FILE_PATTERN.matcher(f.getName()); + if (m.matches()) { + int dstLogId = Integer.parseUnsignedInt(m.group(1), 16); + int srcLogId = Integer.parseUnsignedInt(m.group(2), 16); - logs.add(DirectCompactionEntryLog.recoverLog(srcLogId, dstLogId, ledgerDir, - readBufferSize, maxSaneEntrySize, - nativeIO, allocator, - stats.getReadBlockStats(), - slog)); + logs.add(DirectCompactionEntryLog.recoverLog(srcLogId, dstLogId, ledgerDir, + readBufferSize, maxSaneEntrySize, + nativeIO, allocator, + stats.getReadBlockStats(), + slog)); + } + } } } return logs; diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java index daab3103c52..6bfb6fe69f0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectReader.java @@ -311,7 +311,6 @@ private int sizeInBlock(long offset, int size) { .kv("currentBlockEnd", currentBlockEnd).toString()); } - long blockOffset = offset - currentBlock; long available = currentBlockEnd - offset; checkState(available <= Integer.MAX_VALUE, "Available(%d) must be less than max int", available); return Math.min(size, (int) available); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java index 37a54d087e3..84f33236a85 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -49,7 +49,7 @@ class DirectWriter implements LogWriter { final List> outstandingWrites = new ArrayList>(); Buffer nativeBuffer; long offset; - private static volatile boolean useFallocate = true; + private volatile boolean useFallocate = true; DirectWriter(int id, String filename, diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java index e78e67f0cf5..11e526df7d9 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/LogMetadata.java @@ -135,7 +135,6 @@ static EntryLogMetadata read(LogReader reader) throws IOException { throw new IOException(exMsg("ledgerMap never written").kv("ledgerMapOffset", ledgerMapOffset) .toString()); } - int ledgerCount = Header.extractLedgerCount(header); long offset = ledgerMapOffset; EntryLogMetadata meta = new EntryLogMetadata(reader.logId()); diff --git a/native-io/pom.xml b/native-io/pom.xml index 48f3aefe480..d61ae3dfcd9 100644 --- a/native-io/pom.xml +++ b/native-io/pom.xml @@ -30,6 +30,7 @@ dynamic + -msse4.2 -mpclmul diff --git a/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java b/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java index 667a4008efe..187c98d80f7 100644 --- a/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java +++ b/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java @@ -49,7 +49,7 @@ class NativeIOJni { static { try { if (SystemUtils.IS_OS_MAC_OSX) { - NativeUtils.loadLibraryFromJar("/lib/libnative-io-jni.dylib"); + NativeUtils.loadLibraryFromJar("/lib/libnative-io.jnilib"); } else if (SystemUtils.IS_OS_LINUX) { NativeUtils.loadLibraryFromJar("/lib/libnative-io-jni.so"); } else { From 4b27a8caae05a1a0251d8aed64df6bc766be3d97 Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 16:19:37 +0800 Subject: [PATCH 05/15] fix jnilib not found --- pom.xml | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/pom.xml b/pom.xml index f2fe3598e4f..8075d54f1f8 100644 --- a/pom.xml +++ b/pom.xml @@ -209,6 +209,7 @@ 1 4.0.0 3.0.1 + 3.0.0-M6 @@ -919,8 +920,23 @@ false 1800 ${testRetryCount} + false + -Djava.library.path=target/natives + + org.apache.maven.plugins + maven-failsafe-plugin + ${maven-failsafe-plugin.version} + + + + integration-test + verify + + + + org.apache.maven.plugins maven-javadoc-plugin From f0ab4fb2e1c8c3766bca9e482d614dceef8c2937 Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 16:22:32 +0800 Subject: [PATCH 06/15] fix jnilib not found --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 8075d54f1f8..4aa0cc6b667 100644 --- a/pom.xml +++ b/pom.xml @@ -932,7 +932,6 @@ integration-test - verify From ae9e0e476adc47f2b2af6b482ef2fab5d9aecca3 Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 16:44:57 +0800 Subject: [PATCH 07/15] address comments --- pom.xml | 9 --------- 1 file changed, 9 deletions(-) diff --git a/pom.xml b/pom.xml index 4aa0cc6b667..7f7390275bf 100644 --- a/pom.xml +++ b/pom.xml @@ -920,21 +920,12 @@ false 1800 ${testRetryCount} - false - -Djava.library.path=target/natives org.apache.maven.plugins maven-failsafe-plugin ${maven-failsafe-plugin.version} - - - - integration-test - - - org.apache.maven.plugins From 65e5bf7b6d3b01157416df6dfca908c04d24fb46 Mon Sep 17 00:00:00 2001 From: chenhang Date: Mon, 9 May 2022 23:06:13 +0800 Subject: [PATCH 08/15] fix so lib not found --- .../org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java b/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java index 187c98d80f7..785832b1bc5 100644 --- a/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java +++ b/native-io/src/main/java/org/apache/bookkeeper/common/util/nativeio/NativeIOJni.java @@ -51,7 +51,7 @@ class NativeIOJni { if (SystemUtils.IS_OS_MAC_OSX) { NativeUtils.loadLibraryFromJar("/lib/libnative-io.jnilib"); } else if (SystemUtils.IS_OS_LINUX) { - NativeUtils.loadLibraryFromJar("/lib/libnative-io-jni.so"); + NativeUtils.loadLibraryFromJar("/lib/libnative-io.so"); } else { throw new RuntimeException("OS not supported by Native-IO utils"); } From 46d61af474c27b9fb4fe4c04d2f247ff6365a517 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sun, 12 Jun 2022 19:58:48 +0800 Subject: [PATCH 09/15] address comments --- .../bookie/storage/EntryLogIdsImpl.java | 13 +++++----- .../directentrylogger/DirectWriter.java | 4 +-- .../bookie/storage/TestEntryLogIds.java | 25 ++++++++++--------- 3 files changed, 22 insertions(+), 20 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java index f8df75c7c48..14bacecfbc8 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogIdsImpl.java @@ -32,6 +32,7 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.directentrylogger.Events; import org.apache.bookkeeper.slogger.Slogger; +import org.apache.commons.lang3.tuple.Pair; /** * EntryLogIdsImpl. @@ -77,9 +78,9 @@ private void findLargestGap() throws IOException { currentIds.addAll(compactedLogIdsInDirectory(ledgerDir)); } - int[] gap = findLargestGap(currentIds); - nextId = gap[0]; - maxId = gap[1]; + Pair gap = findLargestGap(currentIds); + nextId = gap.getLeft(); + maxId = gap.getRight(); slog.kv("dirs", ledgerDirsManager.getAllLedgerDirs()) .kv("nextId", nextId) .kv("maxId", maxId) @@ -93,9 +94,9 @@ private void findLargestGap() throws IOException { * Entry logs should be about 1GB in size, so even if the node * stores a PB, there should be only 1000000 entry logs. */ - static int[] findLargestGap(List currentIds) { + static Pair findLargestGap(List currentIds) { if (currentIds.isEmpty()) { - return new int[] { 0, Integer.MAX_VALUE }; + return Pair.of(0, Integer.MAX_VALUE); } Collections.sort(currentIds); @@ -117,7 +118,7 @@ static int[] findLargestGap(List currentIds) { maxIdCandidate = gapEnd; } } - return new int[] { nextIdCandidate, maxIdCandidate }; + return Pair.of(nextIdCandidate, maxIdCandidate); } public static List logIdsInDirectory(File directory) { diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java index 84f33236a85..0c463f5ef40 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -68,7 +68,7 @@ class DirectWriter implements LogWriter { try { fd = nativeIO.open(filename, NativeIO.O_CREAT | NativeIO.O_WRONLY | NativeIO.O_DIRECT, - 00755); + 00644); checkState(fd >= 0, "Open should have thrown exception, fd is invalid : %d", fd); } catch (NativeIOException ne) { throw new IOException(exMsg(ne.getMessage()).kv("file", filename) @@ -204,7 +204,7 @@ public void flush() throws IOException { public void close() throws IOException { synchronized (bufferLock) { if (nativeBuffer != null && nativeBuffer.position() > 0) { - flushBuffer(); + flush(); } } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java index b0a85551f7c..032f575ba42 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java @@ -29,6 +29,7 @@ import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.junit.Assert.assertEquals; import com.google.common.collect.Lists; @@ -39,6 +40,7 @@ import org.apache.bookkeeper.bookie.storage.directentrylogger.DirectEntryLogger; import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.test.TmpDirs; +import org.apache.commons.lang3.tuple.Pair; import org.junit.After; import org.junit.Test; @@ -214,18 +216,17 @@ public void testCompactedLogsConsidered() throws Exception { @Test public void testGapSelection() throws Exception { - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList())), - arrayContaining(0, Integer.MAX_VALUE)); - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(0))), - arrayContaining(1, Integer.MAX_VALUE)); - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(1, 2, 3, 4, 5, 6))), - arrayContaining(7, Integer.MAX_VALUE)); - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE))), - arrayContaining(0, Integer.MAX_VALUE)); - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2))), - arrayContaining(0, Integer.MAX_VALUE / 2)); - assertThat(toObject(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2 - 1))), - arrayContaining(Integer.MAX_VALUE / 2, Integer.MAX_VALUE)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList()), Pair.of(0, Integer.MAX_VALUE)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(0)), + Pair.of(1, Integer.MAX_VALUE)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(1, 2, 3, 4, 5, 6)), + Pair.of(7, Integer.MAX_VALUE)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE)), + Pair.of(0, Integer.MAX_VALUE)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2)), + Pair.of(0, Integer.MAX_VALUE/2)); + assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2 - 1)), + Pair.of(Integer.MAX_VALUE / 2, Integer.MAX_VALUE)); } private static void touchLog(File ledgerDir, int logId) throws Exception { From e197963742e74b0de95ee7662eff863933f480c7 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sun, 12 Jun 2022 20:06:46 +0800 Subject: [PATCH 10/15] format code --- .../org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java index 032f575ba42..e93d2f4f03f 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/TestEntryLogIds.java @@ -24,9 +24,7 @@ import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.makeEntry; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newDirsManager; import static org.apache.bookkeeper.bookie.storage.EntryLogTestUtils.newLegacyEntryLogger; -import static org.apache.commons.lang3.ArrayUtils.toObject; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.arrayContaining; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; @@ -224,7 +222,7 @@ public void testGapSelection() throws Exception { assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE)), Pair.of(0, Integer.MAX_VALUE)); assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2)), - Pair.of(0, Integer.MAX_VALUE/2)); + Pair.of(0, Integer.MAX_VALUE / 2)); assertEquals(EntryLogIdsImpl.findLargestGap(Lists.newArrayList(Integer.MAX_VALUE / 2 - 1)), Pair.of(Integer.MAX_VALUE / 2, Integer.MAX_VALUE)); } From 660aee4afd3306592f212a2f843a173edd842d70 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sun, 19 Jun 2022 14:45:37 +0800 Subject: [PATCH 11/15] add compat test --- .../TestDirectEntryLogger.java | 329 ++++++++++++++++++ 1 file changed, 329 insertions(+) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java index 748acd6d16e..45d61157c35 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java @@ -32,21 +32,28 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; +import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; +import org.apache.bookkeeper.util.DiskChecker; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -54,6 +61,7 @@ /** * TestDirectEntryLogger. */ +@Slf4j public class TestDirectEntryLogger { private final Slogger slog = Slogger.CONSOLE; @@ -104,6 +112,327 @@ curDir, new MockEntryLogIds(), } } + // step1: default is DirectEntryLogger, write entries, read entries + // step2: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written + // by DirectEntryLogger and DefaultEntryLogger + // step3: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by + // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. + // DirectEntryLogger -> DefaultEntryLogge -> DirectEntryLogger. + @Test + public void testCompatFromDirectToDefaultToDirectLogger() throws Exception { + File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + MockEntryLogIds entryLogIds = new MockEntryLogIds(); + + ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); + ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); + ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); + ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); + ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); + ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); + ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); + + long loc1, loc2, loc3, loc4, loc5, loc6, loc7; + + // write entry into DirectEntryLogger + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1.slice()); + loc2 = elog.addEntry(ledgerId1, e2.slice()); + loc3 = elog.addEntry(ledgerId1, e3.slice()); + elog.flush(); + + ByteBuf entry1 = elog.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = elog.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + + entry1.release(); + entry2.release(); + entry3.release(); + } + + // read entry from DefaultEntryLogger + ServerConfiguration conf = new ServerConfiguration(); + LedgerDirsManager dirsMgr = new LedgerDirsManager( + conf, + new File[] { ledgerDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); + EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc4 = entryLogger.addEntry(ledgerId1, e4.slice()); + loc5 = entryLogger.addEntry(ledgerId1, e5.slice()); + entryLogger.flush(); + + ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + + // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc6 = elog.addEntry(ledgerId1, e6.slice()); + loc7 = elog.addEntry(ledgerId1, e7.slice()); + elog.flush(); + + log.info("[hangc] {} - {} - {} - {} - {} - {} - {}", loc1, loc2, loc3, loc4, loc5, loc6, loc7); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + entry3 = elog.readEntry(ledgerId1, 3, loc3); + entry4 = elog.readEntry(ledgerId1, 4, loc4); + entry5 = elog.readEntry(ledgerId1, 5, loc5); + ByteBuf entry6 = elog.readEntry(ledgerId1, 6, loc6); + ByteBuf entry7 = elog.readEntry(ledgerId1, 7, loc7); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + } + + ledgerDir.deleteOnExit(); + + } + + // step1: default is DefaultEntryLogger, write entries and read entries. + // step2: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries both writer + // by DefaultEntryLogger and DirectEntryLogger + // step3: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written + // by DirectEntryLogger and DefaultEntryLogger + // step4: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by + // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. + // DefaultEntryLogger -> DirectEntryLogger -> DefaultEntryLogger -> DirectEntryLogger. + @Test + public void testCompatFromDefaultToDirectToDefaultToDirectLogger() throws Exception { + File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + MockEntryLogIds entryLogIds = new MockEntryLogIds(); + + ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); + ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); + ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); + ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); + ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); + ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); + ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); + ByteBuf e8 = buildEntry(ledgerId1, 8, 1024, "entry-8".getBytes(StandardCharsets.UTF_8)); + ByteBuf e9 = buildEntry(ledgerId1, 9, 1024, "entry-9".getBytes(StandardCharsets.UTF_8)); + + long loc1, loc2, loc3, loc4, loc5, loc6, loc7, loc8, loc9; + + // write e1 and e2 using DefaultEntryLogger + ServerConfiguration conf = new ServerConfiguration(); + LedgerDirsManager dirsMgr = new LedgerDirsManager( + conf, + new File[] { ledgerDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); + EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc1 = entryLogger.addEntry(ledgerId1, e1.slice()); + loc2 = entryLogger.addEntry(ledgerId1, e2.slice()); + entryLogger.flush(); + + ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + + entry1.release(); + entry2.release(); + + // write e3, e4 and e5 using DirectEntryLogger and read all entries. + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc3 = elog.addEntry(ledgerId1, e3.slice()); + loc4 = elog.addEntry(ledgerId1, e4.slice()); + loc5 = elog.addEntry(ledgerId1, e5.slice()); + elog.flush(); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = elog.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = elog.readEntry(ledgerId1, 5, loc5); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + } + + // write e6 and e7 using DefaultEntryLogger and read all entries + entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc6 = entryLogger.addEntry(ledgerId1, e6.slice()); + loc7 = entryLogger.addEntry(ledgerId1, e7.slice()); + entryLogger.flush(); + + entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); + ByteBuf entry6 = entryLogger.readEntry(ledgerId1, 6, loc6); + ByteBuf entry7 = entryLogger.readEntry(ledgerId1, 7, loc7); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + + // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc8 = elog.addEntry(ledgerId1, e8.slice()); + loc9 = elog.addEntry(ledgerId1, e9.slice()); + elog.flush(); + + log.info("[hangc] {} - {} - {} - {} - {} - {} - {}", loc1, loc2, loc3, loc4, loc5, loc6, loc7); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + entry3 = elog.readEntry(ledgerId1, 3, loc3); + entry4 = elog.readEntry(ledgerId1, 4, loc4); + entry5 = elog.readEntry(ledgerId1, 5, loc5); + entry6 = elog.readEntry(ledgerId1, 6, loc6); + entry7 = elog.readEntry(ledgerId1, 7, loc7); + ByteBuf entry8 = elog.readEntry(ledgerId1, 8, loc8); + ByteBuf entry9 = elog.readEntry(ledgerId1, 9, loc9); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + assertEntryEquals(entry8, e8); + assertEntryEquals(entry9, e9); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + entry8.release(); + entry9.release(); + } + + ledgerDir.deleteOnExit(); + } + + private ByteBuf buildEntry(long ledgerId, long entryId, int size, byte[] bytes) { + ByteBuf entry = Unpooled.buffer(size); + entry.writeLong(ledgerId); // ledger id + entry.writeLong(entryId); // entry id + entry.writeBytes(bytes); + return entry; + } + @Test public void testReadLog() throws Exception { File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); From 2f758b7e95dd81b79cbf5e42bccf9fbecfd24ed4 Mon Sep 17 00:00:00 2001 From: chenhang Date: Sun, 19 Jun 2022 14:50:00 +0800 Subject: [PATCH 12/15] add compat test --- .../TestDirectEntryLogger.java | 327 ------------------ .../TestDirectEntryLoggerCompat.java | 322 +++++++++++++++++ 2 files changed, 322 insertions(+), 327 deletions(-) diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java index 45d61157c35..6d9068fb214 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java @@ -32,10 +32,8 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; -import io.netty.buffer.Unpooled; import java.io.File; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -43,17 +41,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; -import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.EntryLogMetadata; -import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; -import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; -import org.apache.bookkeeper.util.DiskChecker; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -112,327 +106,6 @@ curDir, new MockEntryLogIds(), } } - // step1: default is DirectEntryLogger, write entries, read entries - // step2: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written - // by DirectEntryLogger and DefaultEntryLogger - // step3: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by - // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. - // DirectEntryLogger -> DefaultEntryLogge -> DirectEntryLogger. - @Test - public void testCompatFromDirectToDefaultToDirectLogger() throws Exception { - File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); - File curDir = new File(ledgerDir, "current"); - curDir.mkdirs(); - MockEntryLogIds entryLogIds = new MockEntryLogIds(); - - ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); - ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); - ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); - ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); - ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); - ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); - ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); - - long loc1, loc2, loc3, loc4, loc5, loc6, loc7; - - // write entry into DirectEntryLogger - try (EntryLogger elog = new DirectEntryLogger( - curDir, entryLogIds, - new NativeIOImpl(), - ByteBufAllocator.DEFAULT, - MoreExecutors.newDirectExecutorService(), - MoreExecutors.newDirectExecutorService(), - 9000, // max file size (header + size of one entry) - 10 * 1024 * 1024, // max sane entry size - 1024 * 1024, // total write buffer size - 1024 * 1024, // total read buffer size - 64 * 1024, // read buffer size - 1, // numReadThreads - 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { - loc1 = elog.addEntry(ledgerId1, e1.slice()); - loc2 = elog.addEntry(ledgerId1, e2.slice()); - loc3 = elog.addEntry(ledgerId1, e3.slice()); - elog.flush(); - - ByteBuf entry1 = elog.readEntry(ledgerId1, 1, loc1); - ByteBuf entry2 = elog.readEntry(ledgerId1, 2, loc2); - ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - - entry1.release(); - entry2.release(); - entry3.release(); - } - - // read entry from DefaultEntryLogger - ServerConfiguration conf = new ServerConfiguration(); - LedgerDirsManager dirsMgr = new LedgerDirsManager( - conf, - new File[] { ledgerDir }, - new DiskChecker( - conf.getDiskUsageThreshold(), - conf.getDiskUsageWarnThreshold())); - EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); - loc4 = entryLogger.addEntry(ledgerId1, e4.slice()); - loc5 = entryLogger.addEntry(ledgerId1, e5.slice()); - entryLogger.flush(); - - ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); - ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); - ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); - ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); - ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - assertEntryEquals(entry4, e4); - assertEntryEquals(entry5, e5); - - entry1.release(); - entry2.release(); - entry3.release(); - entry4.release(); - entry5.release(); - - // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger - entryLogIds.nextId(); - try (EntryLogger elog = new DirectEntryLogger( - curDir, entryLogIds, - new NativeIOImpl(), - ByteBufAllocator.DEFAULT, - MoreExecutors.newDirectExecutorService(), - MoreExecutors.newDirectExecutorService(), - 9000, // max file size (header + size of one entry) - 10 * 1024 * 1024, // max sane entry size - 1024 * 1024, // total write buffer size - 1024 * 1024, // total read buffer size - 64 * 1024, // read buffer size - 1, // numReadThreads - 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { - loc6 = elog.addEntry(ledgerId1, e6.slice()); - loc7 = elog.addEntry(ledgerId1, e7.slice()); - elog.flush(); - - log.info("[hangc] {} - {} - {} - {} - {} - {} - {}", loc1, loc2, loc3, loc4, loc5, loc6, loc7); - - entry1 = elog.readEntry(ledgerId1, 1, loc1); - entry2 = elog.readEntry(ledgerId1, 2, loc2); - entry3 = elog.readEntry(ledgerId1, 3, loc3); - entry4 = elog.readEntry(ledgerId1, 4, loc4); - entry5 = elog.readEntry(ledgerId1, 5, loc5); - ByteBuf entry6 = elog.readEntry(ledgerId1, 6, loc6); - ByteBuf entry7 = elog.readEntry(ledgerId1, 7, loc7); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - assertEntryEquals(entry4, e4); - assertEntryEquals(entry5, e5); - assertEntryEquals(entry6, e6); - assertEntryEquals(entry7, e7); - - entry1.release(); - entry2.release(); - entry3.release(); - entry4.release(); - entry5.release(); - entry6.release(); - entry7.release(); - } - - ledgerDir.deleteOnExit(); - - } - - // step1: default is DefaultEntryLogger, write entries and read entries. - // step2: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries both writer - // by DefaultEntryLogger and DirectEntryLogger - // step3: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written - // by DirectEntryLogger and DefaultEntryLogger - // step4: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by - // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. - // DefaultEntryLogger -> DirectEntryLogger -> DefaultEntryLogger -> DirectEntryLogger. - @Test - public void testCompatFromDefaultToDirectToDefaultToDirectLogger() throws Exception { - File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); - File curDir = new File(ledgerDir, "current"); - curDir.mkdirs(); - MockEntryLogIds entryLogIds = new MockEntryLogIds(); - - ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); - ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); - ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); - ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); - ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); - ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); - ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); - ByteBuf e8 = buildEntry(ledgerId1, 8, 1024, "entry-8".getBytes(StandardCharsets.UTF_8)); - ByteBuf e9 = buildEntry(ledgerId1, 9, 1024, "entry-9".getBytes(StandardCharsets.UTF_8)); - - long loc1, loc2, loc3, loc4, loc5, loc6, loc7, loc8, loc9; - - // write e1 and e2 using DefaultEntryLogger - ServerConfiguration conf = new ServerConfiguration(); - LedgerDirsManager dirsMgr = new LedgerDirsManager( - conf, - new File[] { ledgerDir }, - new DiskChecker( - conf.getDiskUsageThreshold(), - conf.getDiskUsageWarnThreshold())); - EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); - loc1 = entryLogger.addEntry(ledgerId1, e1.slice()); - loc2 = entryLogger.addEntry(ledgerId1, e2.slice()); - entryLogger.flush(); - - ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); - ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - - entry1.release(); - entry2.release(); - - // write e3, e4 and e5 using DirectEntryLogger and read all entries. - entryLogIds.nextId(); - try (EntryLogger elog = new DirectEntryLogger( - curDir, entryLogIds, - new NativeIOImpl(), - ByteBufAllocator.DEFAULT, - MoreExecutors.newDirectExecutorService(), - MoreExecutors.newDirectExecutorService(), - 9000, // max file size (header + size of one entry) - 10 * 1024 * 1024, // max sane entry size - 1024 * 1024, // total write buffer size - 1024 * 1024, // total read buffer size - 64 * 1024, // read buffer size - 1, // numReadThreads - 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { - loc3 = elog.addEntry(ledgerId1, e3.slice()); - loc4 = elog.addEntry(ledgerId1, e4.slice()); - loc5 = elog.addEntry(ledgerId1, e5.slice()); - elog.flush(); - - entry1 = elog.readEntry(ledgerId1, 1, loc1); - entry2 = elog.readEntry(ledgerId1, 2, loc2); - ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); - ByteBuf entry4 = elog.readEntry(ledgerId1, 4, loc4); - ByteBuf entry5 = elog.readEntry(ledgerId1, 5, loc5); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - assertEntryEquals(entry4, e4); - assertEntryEquals(entry5, e5); - - entry1.release(); - entry2.release(); - entry3.release(); - entry4.release(); - entry5.release(); - } - - // write e6 and e7 using DefaultEntryLogger and read all entries - entryLogger = new DefaultEntryLogger(conf, dirsMgr); - loc6 = entryLogger.addEntry(ledgerId1, e6.slice()); - loc7 = entryLogger.addEntry(ledgerId1, e7.slice()); - entryLogger.flush(); - - entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); - entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); - ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); - ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); - ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); - ByteBuf entry6 = entryLogger.readEntry(ledgerId1, 6, loc6); - ByteBuf entry7 = entryLogger.readEntry(ledgerId1, 7, loc7); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - assertEntryEquals(entry4, e4); - assertEntryEquals(entry5, e5); - assertEntryEquals(entry6, e6); - assertEntryEquals(entry7, e7); - - entry1.release(); - entry2.release(); - entry3.release(); - entry4.release(); - entry5.release(); - entry6.release(); - entry7.release(); - - // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger - entryLogIds.nextId(); - try (EntryLogger elog = new DirectEntryLogger( - curDir, entryLogIds, - new NativeIOImpl(), - ByteBufAllocator.DEFAULT, - MoreExecutors.newDirectExecutorService(), - MoreExecutors.newDirectExecutorService(), - 9000, // max file size (header + size of one entry) - 10 * 1024 * 1024, // max sane entry size - 1024 * 1024, // total write buffer size - 1024 * 1024, // total read buffer size - 64 * 1024, // read buffer size - 1, // numReadThreads - 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { - loc8 = elog.addEntry(ledgerId1, e8.slice()); - loc9 = elog.addEntry(ledgerId1, e9.slice()); - elog.flush(); - - log.info("[hangc] {} - {} - {} - {} - {} - {} - {}", loc1, loc2, loc3, loc4, loc5, loc6, loc7); - - entry1 = elog.readEntry(ledgerId1, 1, loc1); - entry2 = elog.readEntry(ledgerId1, 2, loc2); - entry3 = elog.readEntry(ledgerId1, 3, loc3); - entry4 = elog.readEntry(ledgerId1, 4, loc4); - entry5 = elog.readEntry(ledgerId1, 5, loc5); - entry6 = elog.readEntry(ledgerId1, 6, loc6); - entry7 = elog.readEntry(ledgerId1, 7, loc7); - ByteBuf entry8 = elog.readEntry(ledgerId1, 8, loc8); - ByteBuf entry9 = elog.readEntry(ledgerId1, 9, loc9); - - assertEntryEquals(entry1, e1); - assertEntryEquals(entry2, e2); - assertEntryEquals(entry3, e3); - assertEntryEquals(entry4, e4); - assertEntryEquals(entry5, e5); - assertEntryEquals(entry6, e6); - assertEntryEquals(entry7, e7); - assertEntryEquals(entry8, e8); - assertEntryEquals(entry9, e9); - - entry1.release(); - entry2.release(); - entry3.release(); - entry4.release(); - entry5.release(); - entry6.release(); - entry7.release(); - entry8.release(); - entry9.release(); - } - - ledgerDir.deleteOnExit(); - } - - private ByteBuf buildEntry(long ledgerId, long entryId, int size, byte[] bytes) { - ByteBuf entry = Unpooled.buffer(size); - entry.writeLong(ledgerId); // ledger id - entry.writeLong(entryId); // entry id - entry.writeBytes(bytes); - return entry; - } - @Test public void testReadLog() throws Exception { File ledgerDir = tmpDirs.createNew("logRolling", "ledgers"); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java index 487e94a0711..1d4dc5faffb 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java @@ -31,14 +31,20 @@ import com.google.common.util.concurrent.MoreExecutors; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; +import io.netty.buffer.Unpooled; import java.io.File; +import java.nio.charset.StandardCharsets; +import org.apache.bookkeeper.bookie.DefaultEntryLogger; import org.apache.bookkeeper.bookie.EntryLogMetadata; +import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; +import org.apache.bookkeeper.conf.ServerConfiguration; import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; +import org.apache.bookkeeper.util.DiskChecker; import org.junit.After; import org.junit.Test; @@ -325,5 +331,321 @@ curDir, new MockEntryLogIds(), } } + // step1: default is DirectEntryLogger, write entries, read entries + // step2: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written + // by DirectEntryLogger and DefaultEntryLogger + // step3: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by + // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. + // DirectEntryLogger -> DefaultEntryLogge -> DirectEntryLogger. + @Test + public void testCompatFromDirectToDefaultToDirectLogger() throws Exception { + File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + MockEntryLogIds entryLogIds = new MockEntryLogIds(); + + ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); + ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); + ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); + ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); + ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); + ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); + ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); + + long loc1, loc2, loc3, loc4, loc5, loc6, loc7; + + // write entry into DirectEntryLogger + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc1 = elog.addEntry(ledgerId1, e1.slice()); + loc2 = elog.addEntry(ledgerId1, e2.slice()); + loc3 = elog.addEntry(ledgerId1, e3.slice()); + elog.flush(); + + ByteBuf entry1 = elog.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = elog.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + + entry1.release(); + entry2.release(); + entry3.release(); + } + + // read entry from DefaultEntryLogger + ServerConfiguration conf = new ServerConfiguration(); + LedgerDirsManager dirsMgr = new LedgerDirsManager( + conf, + new File[] { ledgerDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); + EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc4 = entryLogger.addEntry(ledgerId1, e4.slice()); + loc5 = entryLogger.addEntry(ledgerId1, e5.slice()); + entryLogger.flush(); + + ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + + // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc6 = elog.addEntry(ledgerId1, e6.slice()); + loc7 = elog.addEntry(ledgerId1, e7.slice()); + elog.flush(); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + entry3 = elog.readEntry(ledgerId1, 3, loc3); + entry4 = elog.readEntry(ledgerId1, 4, loc4); + entry5 = elog.readEntry(ledgerId1, 5, loc5); + ByteBuf entry6 = elog.readEntry(ledgerId1, 6, loc6); + ByteBuf entry7 = elog.readEntry(ledgerId1, 7, loc7); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + } + + ledgerDir.deleteOnExit(); + + } + + // step1: default is DefaultEntryLogger, write entries and read entries. + // step2: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries both writer + // by DefaultEntryLogger and DirectEntryLogger + // step3: change DirectEntryLogger to DefaultEntryLogger, write entries, and read all entries both written + // by DirectEntryLogger and DefaultEntryLogger + // step4: change DefaultEntryLogger to DirectEntryLogger, write entries, and read all entries written by + // DirectEntryLogger, DefaultEntryLogger and DirectEntryLogger. + // DefaultEntryLogger -> DirectEntryLogger -> DefaultEntryLogger -> DirectEntryLogger. + @Test + public void testCompatFromDefaultToDirectToDefaultToDirectLogger() throws Exception { + File ledgerDir = tmpDirs.createNew("entryCompatTest", "ledgers"); + File curDir = new File(ledgerDir, "current"); + curDir.mkdirs(); + MockEntryLogIds entryLogIds = new MockEntryLogIds(); + + ByteBuf e1 = buildEntry(ledgerId1, 1, 1024, "entry-1".getBytes(StandardCharsets.UTF_8)); + ByteBuf e2 = buildEntry(ledgerId1, 2, 1024, "entry-2".getBytes(StandardCharsets.UTF_8)); + ByteBuf e3 = buildEntry(ledgerId1, 3, 1024, "entry-3".getBytes(StandardCharsets.UTF_8)); + ByteBuf e4 = buildEntry(ledgerId1, 4, 1024, "entry-4".getBytes(StandardCharsets.UTF_8)); + ByteBuf e5 = buildEntry(ledgerId1, 5, 1024, "entry-5".getBytes(StandardCharsets.UTF_8)); + ByteBuf e6 = buildEntry(ledgerId1, 6, 1024, "entry-6".getBytes(StandardCharsets.UTF_8)); + ByteBuf e7 = buildEntry(ledgerId1, 7, 1024, "entry-7".getBytes(StandardCharsets.UTF_8)); + ByteBuf e8 = buildEntry(ledgerId1, 8, 1024, "entry-8".getBytes(StandardCharsets.UTF_8)); + ByteBuf e9 = buildEntry(ledgerId1, 9, 1024, "entry-9".getBytes(StandardCharsets.UTF_8)); + + long loc1, loc2, loc3, loc4, loc5, loc6, loc7, loc8, loc9; + + // write e1 and e2 using DefaultEntryLogger + ServerConfiguration conf = new ServerConfiguration(); + LedgerDirsManager dirsMgr = new LedgerDirsManager( + conf, + new File[] { ledgerDir }, + new DiskChecker( + conf.getDiskUsageThreshold(), + conf.getDiskUsageWarnThreshold())); + EntryLogger entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc1 = entryLogger.addEntry(ledgerId1, e1.slice()); + loc2 = entryLogger.addEntry(ledgerId1, e2.slice()); + entryLogger.flush(); + + ByteBuf entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + ByteBuf entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + + entry1.release(); + entry2.release(); + + // write e3, e4 and e5 using DirectEntryLogger and read all entries. + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc3 = elog.addEntry(ledgerId1, e3.slice()); + loc4 = elog.addEntry(ledgerId1, e4.slice()); + loc5 = elog.addEntry(ledgerId1, e5.slice()); + elog.flush(); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = elog.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = elog.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = elog.readEntry(ledgerId1, 5, loc5); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + } + + // write e6 and e7 using DefaultEntryLogger and read all entries + entryLogger = new DefaultEntryLogger(conf, dirsMgr); + loc6 = entryLogger.addEntry(ledgerId1, e6.slice()); + loc7 = entryLogger.addEntry(ledgerId1, e7.slice()); + entryLogger.flush(); + + entry1 = entryLogger.readEntry(ledgerId1, 1, loc1); + entry2 = entryLogger.readEntry(ledgerId1, 2, loc2); + ByteBuf entry3 = entryLogger.readEntry(ledgerId1, 3, loc3); + ByteBuf entry4 = entryLogger.readEntry(ledgerId1, 4, loc4); + ByteBuf entry5 = entryLogger.readEntry(ledgerId1, 5, loc5); + ByteBuf entry6 = entryLogger.readEntry(ledgerId1, 6, loc6); + ByteBuf entry7 = entryLogger.readEntry(ledgerId1, 7, loc7); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + + // use DirectEntryLogger to read entries written by both DirectEntryLogger and DefaultEntryLogger + entryLogIds.nextId(); + try (EntryLogger elog = new DirectEntryLogger( + curDir, entryLogIds, + new NativeIOImpl(), + ByteBufAllocator.DEFAULT, + MoreExecutors.newDirectExecutorService(), + MoreExecutors.newDirectExecutorService(), + 9000, // max file size (header + size of one entry) + 10 * 1024 * 1024, // max sane entry size + 1024 * 1024, // total write buffer size + 1024 * 1024, // total read buffer size + 64 * 1024, // read buffer size + 1, // numReadThreads + 300, // max fd cache time in seconds + slog, NullStatsLogger.INSTANCE)) { + loc8 = elog.addEntry(ledgerId1, e8.slice()); + loc9 = elog.addEntry(ledgerId1, e9.slice()); + elog.flush(); + + entry1 = elog.readEntry(ledgerId1, 1, loc1); + entry2 = elog.readEntry(ledgerId1, 2, loc2); + entry3 = elog.readEntry(ledgerId1, 3, loc3); + entry4 = elog.readEntry(ledgerId1, 4, loc4); + entry5 = elog.readEntry(ledgerId1, 5, loc5); + entry6 = elog.readEntry(ledgerId1, 6, loc6); + entry7 = elog.readEntry(ledgerId1, 7, loc7); + ByteBuf entry8 = elog.readEntry(ledgerId1, 8, loc8); + ByteBuf entry9 = elog.readEntry(ledgerId1, 9, loc9); + + assertEntryEquals(entry1, e1); + assertEntryEquals(entry2, e2); + assertEntryEquals(entry3, e3); + assertEntryEquals(entry4, e4); + assertEntryEquals(entry5, e5); + assertEntryEquals(entry6, e6); + assertEntryEquals(entry7, e7); + assertEntryEquals(entry8, e8); + assertEntryEquals(entry9, e9); + + entry1.release(); + entry2.release(); + entry3.release(); + entry4.release(); + entry5.release(); + entry6.release(); + entry7.release(); + entry8.release(); + entry9.release(); + } + + ledgerDir.deleteOnExit(); + } + + private ByteBuf buildEntry(long ledgerId, long entryId, int size, byte[] bytes) { + ByteBuf entry = Unpooled.buffer(size); + entry.writeLong(ledgerId); // ledger id + entry.writeLong(entryId); // entry id + entry.writeBytes(bytes); + return entry; + } } From b221412c3523e94d7d1b07b41d7a14e9456b2e44 Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 22 Jun 2022 10:55:19 +0800 Subject: [PATCH 13/15] address comments --- .../DirectEntryLoggerStats.java | 10 +-- .../directentrylogger/DirectWriter.java | 82 +++++++++---------- 2 files changed, 43 insertions(+), 49 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java index 385766702c0..4f53c9a3367 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java @@ -56,7 +56,7 @@ class DirectEntryLoggerStats { help = "Operation stats of reading entries from the entry log", parent = BOOKIE_READ_ENTRY ) - private final ThreadLocal readEntryStats; + private static ThreadLocal readEntryStats; @StatsDoc( name = FLUSH, @@ -74,25 +74,25 @@ class DirectEntryLoggerStats { name = READ_BLOCK, help = "Stats for reading blocks from disk" ) - private final ThreadLocal readBlockStats; + private static ThreadLocal readBlockStats; @StatsDoc( name = READER_OPEN, help = "Stats for reader open operations" ) - private final ThreadLocal openReaderStats; + private static ThreadLocal openReaderStats; @StatsDoc( name = READER_CLOSE, help = "Stats for reader close operations" ) - private final ThreadLocal closeReaderStats; + private static ThreadLocal closeReaderStats; @StatsDoc( name = CACHED_READER_SERVED_CLOSED, help = "Stats for cached readers being served closed" ) - private final ThreadLocal cachedReadersServedClosed; + private static ThreadLocal cachedReadersServedClosed; DirectEntryLoggerStats(StatsLogger stats) { addEntryStats = stats.getOpStatsLogger(ADD_ENTRY); diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java index 0c463f5ef40..4b343aabec0 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; @@ -113,32 +114,47 @@ public void writeAt(long offset, ByteBuf buf) throws IOException { Buffer.ALIGNMENT, buf.readableBytes()); Buffer tmpBuffer = bufferPool.acquire(); int bytesToWrite = buf.readableBytes(); + if (bytesToWrite <= 0) { + return; + } + tmpBuffer.reset(); tmpBuffer.writeByteBuf(buf); Future f = writeExecutor.submit(() -> { - try { - int ret = nativeIO.pwrite(fd, tmpBuffer.pointer(), bytesToWrite, offset); - if (ret != bytesToWrite) { - throw new IOException(exMsg("Incomplete write") - .kv("filename", filename) - .kv("writeSize", bytesToWrite) - .kv("bytesWritten", ret) - .kv("offset", offset).toString()); - } - } catch (NativeIOException ne) { - throw new IOException(exMsg("Write error") - .kv("filename", filename) - .kv("writeSize", bytesToWrite) - .kv("errno", ne.getErrno()) - .kv("offset", offset).toString()); - } finally { - bufferPool.release(tmpBuffer); - } - return null; + writeByteBuf(tmpBuffer, bytesToWrite, offset); + return null; }); addOutstandingWrite(f); } + private void writeByteBuf(Buffer buffer, int bytesToWrite, long offsetToWrite) throws IOException{ + try { + if (bytesToWrite <= 0) { + return; + } + int ret = nativeIO.pwrite(fd, buffer.pointer(), bytesToWrite, offsetToWrite); + if (ret != bytesToWrite) { + throw new IOException(exMsg("Incomplete write") + .kv("filename", filename) + .kv("pointer", buffer.pointer()) + .kv("offset", offsetToWrite) + .kv("writeSize", bytesToWrite) + .kv("bytesWritten", ret) + .toString()); + } + } catch (NativeIOException ne) { + throw new IOException(exMsg("Write error") + .kv("filename", filename) + .kv("offset", offsetToWrite) + .kv("writeSize", bytesToWrite) + .kv("pointer", buffer.pointer()) + .kv("errno", ne.getErrno()) + .toString()); + } finally { + bufferPool.release(buffer); + } + } + @Override public int writeDelimited(ByteBuf buf) throws IOException { synchronized (bufferLock) { @@ -280,31 +296,9 @@ private void flushBuffer() throws IOException { offset += bytesToWrite; Future f = writeExecutor.submit(() -> { - try { - if (bytesToWrite <= 0) { - return null; - } - int ret = nativeIO.pwrite(fd, bufferToFlush.pointer(), bytesToWrite, offsetToWrite); - if (ret != bytesToWrite) { - throw new IOException(exMsg("Incomplete write") - .kv("filename", filename) - .kv("pointer", bufferToFlush.pointer()) - .kv("offset", offsetToWrite) - .kv("writeSize", bytesToWrite) - .kv("bytesWritten", ret).toString()); - } - } catch (NativeIOException ne) { - throw new IOException(exMsg(ne.getMessage()) - .kv("filename", filename) - .kv("offset", offsetToWrite) - .kv("writeSize", bytesToWrite) - .kv("pointer", bufferToFlush.pointer()) - .kv("errno", ne.getErrno()).toString(), ne); - } finally { - bufferPool.release(bufferToFlush); - } - return null; - }); + writeByteBuf(bufferToFlush, bytesToWrite, offsetToWrite); + return null; + }); addOutstandingWrite(f); // must acquire after triggering the write From 8a4d0da23144bdd61bfbcc4d62bbdc75dbbf6969 Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 22 Jun 2022 13:06:08 +0800 Subject: [PATCH 14/15] fix findbugs failed --- .../DirectEntryLoggerStats.java | 35 ++++++++++--------- .../directentrylogger/DirectWriter.java | 1 - 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java index 4f53c9a3367..bfb602a6a39 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java @@ -99,43 +99,46 @@ class DirectEntryLoggerStats { flushStats = stats.getOpStatsLogger(FLUSH); writerFlushStats = stats.getOpStatsLogger(WRITER_FLUSH); + setStats(stats); + } + private synchronized static void setStats(StatsLogger stats) { readEntryStats = new ThreadLocal() { - @Override - public OpStatsLogger initialValue() { - return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + @Override + public OpStatsLogger initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) .getOpStatsLogger(READ_ENTRY); - } - }; + } + }; readBlockStats = new ThreadLocal() { - @Override - public OpStatsLogger initialValue() { - return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) + @Override + public OpStatsLogger initialValue() { + return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) .getOpStatsLogger(READ_BLOCK); - } - }; + } + }; - openReaderStats = new ThreadLocal() { + DirectEntryLoggerStats.openReaderStats = new ThreadLocal() { @Override public Counter initialValue() { return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) - .getCounter(READER_OPEN); + .getCounter(READER_OPEN); } }; - closeReaderStats = new ThreadLocal() { + DirectEntryLoggerStats.closeReaderStats = new ThreadLocal() { @Override public Counter initialValue() { return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) - .getCounter(READER_CLOSE); + .getCounter(READER_CLOSE); } }; - cachedReadersServedClosed = new ThreadLocal() { + DirectEntryLoggerStats.cachedReadersServedClosed = new ThreadLocal() { @Override public Counter initialValue() { return stats.scopeLabel("thread", String.valueOf(Thread.currentThread().getId())) - .getCounter(CACHED_READER_SERVED_CLOSED); + .getCounter(CACHED_READER_SERVED_CLOSED); } }; } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java index 4b343aabec0..13cd7c34f28 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -30,7 +30,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; import java.util.concurrent.Future; From 726d359f10a7630d2a34e275c53b0dd6476e4d01 Mon Sep 17 00:00:00 2001 From: chenhang Date: Wed, 22 Jun 2022 13:11:31 +0800 Subject: [PATCH 15/15] format code --- .../storage/directentrylogger/DirectEntryLoggerStats.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java index bfb602a6a39..b34c1ca5c5e 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLoggerStats.java @@ -102,7 +102,7 @@ class DirectEntryLoggerStats { setStats(stats); } - private synchronized static void setStats(StatsLogger stats) { + private static synchronized void setStats(StatsLogger stats) { readEntryStats = new ThreadLocal() { @Override public OpStatsLogger initialValue() {