diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ExceptionMessageHelper.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ExceptionMessageHelper.java
new file mode 100644
index 00000000000..9fa925d8038
--- /dev/null
+++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/util/ExceptionMessageHelper.java
@@ -0,0 +1,49 @@
+/**
+ * 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.common.util;
+
+/**
+ * Utility to make it easier to add context to exception messages.
+ */
+public class ExceptionMessageHelper {
+ public StringBuilder sb = new StringBuilder();
+ private boolean firstKV = true;
+
+ public static ExceptionMessageHelper exMsg(String msg) {
+ return new ExceptionMessageHelper(msg);
+ }
+
+ ExceptionMessageHelper(String msg) {
+ sb.append(msg).append("(");
+ }
+
+ public ExceptionMessageHelper kv(String key, Object value) {
+ if (firstKV) {
+ firstKV = false;
+ } else {
+ sb.append(",");
+ }
+ sb.append(key).append("=").append(value.toString());
+ return this;
+ }
+
+ public String toString() {
+ return sb.append(")").toString();
+ }
+}
diff --git a/bookkeeper-dist/all/build.gradle b/bookkeeper-dist/all/build.gradle
index 486eede5a84..3eaa1b23fe1 100644
--- a/bookkeeper-dist/all/build.gradle
+++ b/bookkeeper-dist/all/build.gradle
@@ -68,7 +68,7 @@ def depLicences = [
"scala-library-2.11.7/LICENSE.md",
"scala-parser-combinators_2.11-1.0.4/LICENSE.md",
"scala-reflect-2.11.8/LICENSE.md",
- "slf4j-1.7.32/LICENSE.txt",
+ "slf4j-1.7.36/LICENSE.txt",
]
distributions {
diff --git a/bookkeeper-dist/bkctl/build.gradle b/bookkeeper-dist/bkctl/build.gradle
index 23a4aa07853..35de98df658 100644
--- a/bookkeeper-dist/bkctl/build.gradle
+++ b/bookkeeper-dist/bkctl/build.gradle
@@ -54,7 +54,7 @@ def depLicences = [
"protobuf-3.14.0/LICENSE",
"protobuf-3.12.0/LICENSE",
"reactivestreams-1.0.3/LICENSE",
- "slf4j-1.7.32/LICENSE.txt",
+ "slf4j-1.7.36/LICENSE.txt",
]
distributions {
diff --git a/bookkeeper-dist/server/build.gradle b/bookkeeper-dist/server/build.gradle
index 752bccf0e8d..dfbdd3cada5 100644
--- a/bookkeeper-dist/server/build.gradle
+++ b/bookkeeper-dist/server/build.gradle
@@ -58,7 +58,7 @@ def depLicences = [
"protobuf-3.14.0/LICENSE",
"protobuf-3.12.0/LICENSE",
"reactivestreams-1.0.3/LICENSE",
- "slf4j-1.7.32/LICENSE.txt",
+ "slf4j-1.7.36/LICENSE.txt",
]
distributions {
main {
diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml
index ad85bf692f6..c822a6ce68a 100644
--- a/bookkeeper-dist/src/assemble/bin-all.xml
+++ b/bookkeeper-dist/src/assemble/bin-all.xml
@@ -67,7 +67,7 @@
scala-library-2.11.7/LICENSE.mdscala-parser-combinators_2.11-1.0.4/LICENSE.mdscala-reflect-2.11.8/LICENSE.md
- slf4j-1.7.32/LICENSE.txt
+ slf4j-1.7.36/LICENSE.txt644
diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml
index a910c267f68..6724a96f4dd 100644
--- a/bookkeeper-dist/src/assemble/bin-server.xml
+++ b/bookkeeper-dist/src/assemble/bin-server.xml
@@ -57,7 +57,7 @@
protobuf-3.14.0/LICENSEprotobuf-3.12.0/LICENSEreactivestreams-1.0.3/LICENSE
- slf4j-1.7.32/LICENSE.txt
+ slf4j-1.7.36/LICENSE.txt644
diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml
index 6308ecb10f3..a49bce22a53 100644
--- a/bookkeeper-dist/src/assemble/bkctl.xml
+++ b/bookkeeper-dist/src/assemble/bkctl.xml
@@ -74,7 +74,7 @@
protobuf-3.14.0/LICENSEprotobuf-3.12.0/LICENSEreactivestreams-1.0.3/LICENSE
- slf4j-1.7.32/LICENSE.txt
+ slf4j-1.7.36/LICENSE.txt644
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index d774144b4aa..1b4a460fbcf 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -648,11 +648,11 @@ Bundled as lib/javax.servlet-javax.servlet-api-4.0.0.jar
Source available at https://github.com/javaee/servlet-spec/tree/4.0.0
------------------------------------------------------------------------------------
This product bundles Simple Logging Facade for Java, which is available under a
-MIT license. For details, see deps/slf4j-1.7.32/LICENSE.txt.
+MIT license. For details, see deps/slf4j-1.7.36/LICENSE.txt.
Bundled as
- - lib/org.slf4j-slf4j-api-1.7.32.jar
-Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.32
+ - lib/org.slf4j-slf4j-api-1.7.36.jar
+Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.36
------------------------------------------------------------------------------------
This product bundles the Google Auth Library, which is available under a "3-clause BSD"
license. For details, see deps/google-auth-library-credentials-0.20.0/LICENSE
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
index 36fe9a6b6dd..925ea892066 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
@@ -574,11 +574,11 @@ Source available at https://github.com/protocolbuffers/protobuf/tree/v3.19.2
For details, see deps/protobuf-3.12.0/LICENSE.
------------------------------------------------------------------------------------
This product bundles Simple Logging Facade for Java, which is available under a
-MIT license. For details, see deps/slf4j-1.7.32/LICENSE.txt.
+MIT license. For details, see deps/slf4j-1.7.36/LICENSE.txt.
Bundled as
- - lib/org.slf4j-slf4j-api-1.7.32.jar
-Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.32
+ - lib/org.slf4j-slf4j-api-1.7.36.jar
+Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.36
------------------------------------------------------------------------------------
This product bundles the Google Auth Library, which is available under a "3-clause BSD"
license. For details, see deps/google-auth-library-credentials-0.20.0/LICENSE
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index 1bf8f1f30f2..0943e8ac0ad 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -640,11 +640,11 @@ Bundled as lib/javax.servlet-javax.servlet-api-4.0.0.jar
Source available at https://github.com/javaee/servlet-spec/tree/4.0.0
------------------------------------------------------------------------------------
This product bundles Simple Logging Facade for Java, which is available under a
-MIT license. For details, see deps/slf4j-1.7.32/LICENSE.txt.
+MIT license. For details, see deps/slf4j-1.7.36/LICENSE.txt.
Bundled as
- - lib/org.slf4j-slf4j-api-1.7.32.jar
-Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.32
+ - lib/org.slf4j-slf4j-api-1.7.36.jar
+Source available at https://github.com/qos-ch/slf4j/tree/v_1.7.36
------------------------------------------------------------------------------------
This product bundles the Google Auth Library, which is available under a "3-clause BSD"
license. For details, see deps/google-auth-library-credentials-0.20.0/LICENSE
diff --git a/bookkeeper-server/build.gradle b/bookkeeper-server/build.gradle
index 2d48d6b2779..4148a500392 100644
--- a/bookkeeper-server/build.gradle
+++ b/bookkeeper-server/build.gradle
@@ -27,10 +27,13 @@ dependencies {
implementation project(':bookkeeper-common-allocator')
implementation project(':bookkeeper-http:http-server')
implementation project(':bookkeeper-proto')
+ implementation project(':bookkeeper-slogger:api')
+ implementation project(':bookkeeper-slogger:slf4j')
implementation project(':bookkeeper-stats')
implementation project(':bookkeeper-tools-framework')
implementation project(':circe-checksum')
implementation project(':cpu-affinity')
+ implementation project(':native-io')
compileOnly depLibs.lombok
compileOnly depLibs.spotbugsAnnotations
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 57ec8978cc0..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);
}
@@ -57,7 +60,10 @@ public AbstractLogCompactor(ServerConfiguration conf, LogRemovalListener logRemo
*/
public void cleanUpAndRecover() {}
- static class Throttler {
+ /**
+ * class Throttler.
+ */
+ public static class Throttler {
private final RateLimiter rateLimiter;
private final boolean isThrottleByBytes;
@@ -68,7 +74,7 @@ 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/CompactableLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CompactableLedgerStorage.java
index 9a5a0abbcba..982d4e0a32e 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CompactableLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CompactableLedgerStorage.java
@@ -28,11 +28,6 @@
*/
public interface CompactableLedgerStorage extends LedgerStorage {
- /**
- * @return the EntryLogger used by the ledger storage
- */
- EntryLogger getEntryLogger();
-
/**
* Get an iterator over a range of ledger ids stored in the bookie.
*
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/EntryLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java
index 98f4960549f..5843474b4a7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogCompactor.java
@@ -27,7 +27,10 @@
import java.util.ArrayList;
import java.util.List;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.conf.ServerConfiguration;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,13 +43,13 @@ public class EntryLogCompactor extends AbstractLogCompactor {
private static final Logger LOG = LoggerFactory.getLogger(EntryLogCompactor.class);
final CompactionScannerFactory scannerFactory = new CompactionScannerFactory();
- final EntryLogger entryLogger;
+ final EntryLoggerIface entryLogger;
final CompactableLedgerStorage ledgerStorage;
private final int maxOutstandingRequests;
public EntryLogCompactor(
ServerConfiguration conf,
- EntryLogger entryLogger,
+ EntryLoggerIface entryLogger,
CompactableLedgerStorage ledgerStorage,
LogRemovalListener logRemover) {
super(conf, logRemover);
@@ -82,9 +85,9 @@ public boolean compact(EntryLogMetadata entryLogMeta) {
class CompactionScannerFactory {
List offsets = new ArrayList();
- EntryLogger.EntryLogScanner newScanner(final EntryLogMetadata meta) {
+ EntryLogScanner newScanner(final EntryLogMetadata meta) {
- return new EntryLogger.EntryLogScanner() {
+ return new EntryLogScanner() {
@Override
public boolean accept(long ledgerId) {
return meta.containsLedger(ledgerId);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadataMap.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadataMap.java
index 88f6ce53986..f9d41d2ff2a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadataMap.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogMetadataMap.java
@@ -22,7 +22,6 @@
package org.apache.bookkeeper.bookie;
import java.io.Closeable;
-import java.io.IOException;
import java.util.function.BiConsumer;
import org.apache.bookkeeper.bookie.BookieException.EntryLogMetadataMapException;
@@ -37,7 +36,7 @@ public interface EntryLogMetadataMap extends Closeable {
*
* @param entryLogId
* @return
- * @throws IOException
+ * @throws EntryLogMetadataMapException
*/
boolean containsKey(long entryLogId) throws EntryLogMetadataMapException;
@@ -46,7 +45,7 @@ public interface EntryLogMetadataMap extends Closeable {
*
* @param entryLogId
* @param entryLogMeta
- * @throws IOException
+ * @throws EntryLogMetadataMapException
*/
void put(long entryLogId, EntryLogMetadata entryLogMeta) throws EntryLogMetadataMapException;
@@ -55,7 +54,7 @@ public interface EntryLogMetadataMap extends Closeable {
* have been processed or the action throws an exception.
*
* @param action
- * @throws IOException
+ * @throws EntryLogMetadataMapException
*/
void forEach(BiConsumer action) throws EntryLogMetadataMapException;
@@ -63,7 +62,7 @@ public interface EntryLogMetadataMap extends Closeable {
* Removes entryLogMetadata record from the map.
*
* @param entryLogId
- * @throws IOException
+ * @throws EntryLogMetadataMapException
*/
void remove(long entryLogId) throws EntryLogMetadataMapException;
@@ -71,8 +70,24 @@ public interface EntryLogMetadataMap extends Closeable {
* Returns number of entryLogMetadata records presents into the map.
*
* @return
- * @throws IOException
+ * @throws EntryLogMetadataMapException
*/
int size() throws EntryLogMetadataMapException;
+ /**
+ * Returns true iff there are no elements in the map.
+ *
+ * @return
+ */
+ default boolean isEmpty() throws EntryLogMetadataMapException {
+ return size() == 0;
+ }
+
+ /**
+ * Clear all records from the map.
+ * For unit tests.
+ *
+ * @throws EntryLogMetadataMapException
+ */
+ void clear() throws EntryLogMetadataMapException;
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
index 8be995a19c6..c9bb72bacee 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/EntryLogger.java
@@ -45,7 +45,9 @@
import java.nio.channels.AsynchronousCloseException;
import java.nio.channels.FileChannel;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -54,14 +56,20 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.regex.Pattern;
+import org.apache.bookkeeper.bookie.storage.CompactionEntryLog;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.bookkeeper.util.HardLink;
import org.apache.bookkeeper.util.IOUtils;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap.BiConsumerLong;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -72,7 +80,7 @@
* the actual ledger entry. The entry log files created by this class are
* identified by a long.
*/
-public class EntryLogger {
+public class EntryLogger implements EntryLoggerIface {
private static final Logger LOG = LoggerFactory.getLogger(EntryLogger.class);
static final long UNASSIGNED_LEDGERID = -1L;
// log file suffix
@@ -285,33 +293,6 @@ private static class Header {
private final ByteBufAllocator allocator;
final ServerConfiguration conf;
- /**
- * Scan entries in a entry log file.
- */
- public interface EntryLogScanner {
- /**
- * Tests whether or not the entries belongs to the specified ledger
- * should be processed.
- *
- * @param ledgerId
- * Ledger ID.
- * @return true if and only the entries of the ledger should be scanned.
- */
- boolean accept(long ledgerId);
-
- /**
- * Process an entry.
- *
- * @param ledgerId
- * Ledger ID.
- * @param offset
- * File offset of this entry.
- * @param entry
- * Entry ByteBuf
- * @throws IOException
- */
- void process(long ledgerId, long offset, ByteBuf entry) throws IOException;
- }
/**
* Entry Log Listener.
@@ -472,35 +453,25 @@ public BufferedReadChannel getFromChannels(long logId) {
return logid2Channel.get().get(logId);
}
- /**
- * Get the least unflushed log id. Garbage collector thread should not process
- * unflushed entry log file.
- *
- * @return least unflushed log id.
- */
+ @VisibleForTesting
long getLeastUnflushedLogId() {
return recentlyCreatedEntryLogsStatus.getLeastUnflushedLogId();
}
- /**
- * Get the last log id created so far. If entryLogPerLedger is enabled, the Garbage Collector
- * process needs to look beyond the least unflushed entry log file, as there may be entry logs
- * ready to be garbage collected.
- *
- * @return last entry log id created.
- */
- long getLastLogId() {
- return recentlyCreatedEntryLogsStatus.getLastLogId();
- }
-
- /**
- * Returns whether the current log id exists and has been rotated already.
- *
- * @param entryLogId EntryLog id to check.
- * @return Whether the given entryLogId exists and has been rotated.
- */
- boolean isFlushedEntryLog(Long entryLogId) {
- return recentlyCreatedEntryLogsStatus.isFlushedEntryLog(entryLogId);
+ @Override
+ public Set getFlushedLogIds() {
+ Set logIds = new HashSet<>();
+ synchronized (recentlyCreatedEntryLogsStatus) {
+ for (File dir : ledgerDirsManager.getAllLedgerDirs()) {
+ for (File f : dir.listFiles(file -> file.getName().endsWith(".log"))) {
+ long logId = fileName2LogId(f.getName());
+ if (recentlyCreatedEntryLogsStatus.isFlushedLogId(logId)) {
+ logIds.add(logId);
+ }
+ }
+ }
+ }
+ return logIds;
}
long getPreviousAllocatedEntryLogId() {
@@ -510,7 +481,7 @@ long getPreviousAllocatedEntryLogId() {
/**
* Get the current log file for compaction.
*/
- File getCurCompactionLogFile() {
+ private File getCurCompactionLogFile() {
synchronized (compactionLogLock) {
if (compactionLogChannel == null) {
return null;
@@ -544,7 +515,8 @@ EntryLoggerAllocator getEntryLoggerAllocator() {
* @param entryLogId
* Entry Log File Id
*/
- protected boolean removeEntryLog(long entryLogId) {
+ @Override
+ public boolean removeEntryLog(long entryLogId) {
removeFromChannelsAndClose(entryLogId);
File entryLogFile;
try {
@@ -610,6 +582,7 @@ void checkpoint() throws IOException {
entryLogManager.checkpoint();
}
+ @Override
public void flush() throws IOException {
entryLogManager.flush();
}
@@ -618,14 +591,11 @@ long addEntry(long ledger, ByteBuffer entry) throws IOException {
return entryLogManager.addEntry(ledger, Unpooled.wrappedBuffer(entry), true);
}
- long addEntry(long ledger, ByteBuf entry) throws IOException {
+ @Override
+ public long addEntry(long ledger, ByteBuf entry) throws IOException {
return entryLogManager.addEntry(ledger, entry, true);
}
- public long addEntry(long ledger, ByteBuf entry, boolean rollLog) throws IOException {
- return entryLogManager.addEntry(ledger, entry, rollLog);
- }
-
private final FastThreadLocal sizeBuffer = new FastThreadLocal() {
@Override
protected ByteBuf initialValue() throws Exception {
@@ -634,7 +604,7 @@ protected ByteBuf initialValue() throws Exception {
}
};
- long addEntryForCompaction(long ledgerId, ByteBuf entry) throws IOException {
+ private long addEntryForCompaction(long ledgerId, ByteBuf entry) throws IOException {
synchronized (compactionLogLock) {
int entrySize = entry.readableBytes() + 4;
if (compactionLogChannel == null) {
@@ -653,7 +623,7 @@ long addEntryForCompaction(long ledgerId, ByteBuf entry) throws IOException {
}
}
- void flushCompactionLog() throws IOException {
+ private void flushCompactionLog() throws IOException {
synchronized (compactionLogLock) {
if (compactionLogChannel != null) {
compactionLogChannel.appendLedgersMap();
@@ -671,7 +641,7 @@ void flushCompactionLog() throws IOException {
}
}
- void createNewCompactionLog() throws IOException {
+ private void createNewCompactionLog() throws IOException {
synchronized (compactionLogLock) {
if (compactionLogChannel == null) {
compactionLogChannel = entryLogManager.createNewLogForCompaction();
@@ -683,7 +653,7 @@ void createNewCompactionLog() throws IOException {
* Remove the current compaction log, usually invoked when compaction failed and
* we need to do some clean up to remove the compaction log file.
*/
- void removeCurCompactionLog() {
+ private void removeCurCompactionLog() {
synchronized (compactionLogLock) {
if (compactionLogChannel != null) {
if (!compactionLogChannel.getLogFile().delete()) {
@@ -834,8 +804,20 @@ private void validateEntry(long ledgerId, long entryId, long entryLogId, long po
}
}
- public ByteBuf internalReadEntry(long ledgerId, long entryId, long location, boolean validateEntry)
- throws IOException {
+ @Override
+ public ByteBuf readEntry(long ledgerId, long entryId, long entryLocation)
+ throws IOException, Bookie.NoEntryException {
+ return internalReadEntry(ledgerId, entryId, entryLocation, true /* validateEntry */);
+ }
+
+ @Override
+ public ByteBuf readEntry(long location) throws IOException, Bookie.NoEntryException {
+ return internalReadEntry(location, -1L, -1L, false /* validateEntry */);
+ }
+
+
+ private ByteBuf internalReadEntry(long ledgerId, long entryId, long location, boolean validateEntry)
+ throws IOException, Bookie.NoEntryException {
long entryLogId = logIdForOffset(location);
long pos = posForOffset(location);
@@ -877,10 +859,6 @@ public ByteBuf internalReadEntry(long ledgerId, long entryId, long location, boo
return data;
}
- public ByteBuf readEntry(long ledgerId, long entryId, long location) throws IOException, Bookie.NoEntryException {
- return internalReadEntry(ledgerId, entryId, location, true /* validateEntry */);
- }
-
/**
* Read the header of an entry log.
*/
@@ -932,7 +910,8 @@ private BufferedReadChannel getChannelForLogId(long entryLogId) throws IOExcepti
/**
* Whether the log file exists or not.
*/
- boolean logExists(long logId) {
+ @Override
+ public boolean logExists(long logId) {
for (File d : ledgerDirsManager.getAllLedgerDirs()) {
File f = new File(d, Long.toHexString(logId) + ".log");
if (f.exists()) {
@@ -988,6 +967,7 @@ private File findFile(long logId) throws FileNotFoundException {
* @param scanner Entry Log Scanner
* @throws IOException
*/
+ @Override
public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOException {
// Buffer where to read the entrySize (4 bytes) and the ledgerId (8 bytes)
ByteBuf headerBuffer = Unpooled.buffer(4 + 8);
@@ -1019,11 +999,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;
@@ -1031,11 +1017,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) {
@@ -1054,19 +1035,6 @@ public void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOExce
}
}
- public EntryLogMetadata getEntryLogMetadata(long entryLogId) throws IOException {
- // First try to extract the EntryLogMetadata from the index, if there's no index then fallback to scanning the
- // entry log
- try {
- return extractEntryLogMetadataFromIndex(entryLogId);
- } catch (Exception e) {
- LOG.info("Failed to get ledgers map index from: {}.log : {}", entryLogId, e.getMessage());
-
- // Fall-back to scanning
- return extractEntryLogMetadataByScanning(entryLogId);
- }
- }
-
public EntryLogMetadata getEntryLogMetadata(long entryLogId, AbstractLogCompactor.Throttler throttler)
throws IOException {
// First try to extract the EntryLogMetadata from the index, if there's no index then fallback to scanning the
@@ -1113,7 +1081,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);
@@ -1164,10 +1134,6 @@ EntryLogMetadata extractEntryLogMetadataFromIndex(long entryLogId) throws IOExce
return meta;
}
- private EntryLogMetadata extractEntryLogMetadataByScanning(long entryLogId) throws IOException {
- return extractEntryLogMetadataByScanning(entryLogId, null);
- }
-
private EntryLogMetadata extractEntryLogMetadataByScanning(long entryLogId,
AbstractLogCompactor.Throttler throttler)
throws IOException {
@@ -1178,7 +1144,7 @@ private EntryLogMetadata extractEntryLogMetadataByScanning(long entryLogId,
@Override
public void process(long ledgerId, long offset, ByteBuf entry) throws IOException {
if (throttler != null) {
- throttler.acquire(conf.getIsThrottleByBytes() ? entry.readableBytes() : 1);
+ throttler.acquire(entry.readableBytes());
}
// add new entry size of a ledger to entry log meta
meta.addLedgerSize(ledgerId, entry.readableBytes() + 4);
@@ -1199,7 +1165,8 @@ public boolean accept(long ledgerId) {
/**
* Shutdown method to gracefully stop entry logger.
*/
- public void shutdown() {
+ @Override
+ public void close() {
// since logChannel is buffered channel, do flush when shutting down
LOG.info("Stopping EntryLogger");
try {
@@ -1294,13 +1261,179 @@ synchronized long getLeastUnflushedLogId() {
return leastUnflushedLogId;
}
- synchronized long getLastLogId() {
- return !entryLogsStatusMap.isEmpty() ? entryLogsStatusMap.lastKey() : 0;
+ synchronized boolean isFlushedLogId(long entryLogId) {
+ return entryLogsStatusMap.getOrDefault(entryLogId, Boolean.FALSE) || entryLogId < leastUnflushedLogId;
+ }
+ }
+
+ @Override
+ public CompactionEntryLog newCompactionLog(long logToCompact) throws IOException {
+ createNewCompactionLog();
+
+ File compactingLogFile = getCurCompactionLogFile();
+ long compactionLogId = fileName2LogId(compactingLogFile.getName());
+ File compactedLogFile = compactedLogFileFromCompacting(compactingLogFile, logToCompact);
+ File finalLogFile = new File(compactingLogFile.getParentFile(),
+ compactingLogFile.getName().substring(0,
+ compactingLogFile.getName().indexOf(".log") + 4));
+ return new EntryLoggerCompactionEntryLog(
+ compactionLogId, logToCompact, compactingLogFile, compactedLogFile, finalLogFile);
+
+ }
+
+ private class EntryLoggerCompactionEntryLog implements CompactionEntryLog {
+ private final long compactionLogId;
+ private final long logIdToCompact;
+ private final File compactingLogFile;
+ private final File compactedLogFile;
+ private final File finalLogFile;
+
+ EntryLoggerCompactionEntryLog(long compactionLogId, long logIdToCompact,
+ File compactingLogFile,
+ File compactedLogFile,
+ File finalLogFile) {
+ this.compactionLogId = compactionLogId;
+ this.logIdToCompact = logIdToCompact;
+ this.compactingLogFile = compactingLogFile;
+ this.compactedLogFile = compactedLogFile;
+ this.finalLogFile = finalLogFile;
+ }
+
+ @Override
+ public long addEntry(long ledgerId, ByteBuf entry) throws IOException {
+ return addEntryForCompaction(ledgerId, entry);
+ }
+ @Override
+ public void scan(EntryLogScanner scanner) throws IOException {
+ scanEntryLog(compactionLogId, scanner);
+ }
+ @Override
+ public void flush() throws IOException {
+ flushCompactionLog();
+ }
+ @Override
+ public void abort() {
+ removeCurCompactionLog();
+ if (compactedLogFile.exists()) {
+ compactedLogFile.delete();
+ }
+ }
+
+ @Override
+ public void markCompacted() throws IOException {
+ if (compactingLogFile.exists()) {
+ if (!compactedLogFile.exists()) {
+ HardLink.createHardLink(compactingLogFile, compactedLogFile);
+ }
+ } else {
+ throw new IOException("Compaction log doesn't exist any more after flush: " + compactingLogFile);
+ }
+ removeCurCompactionLog();
+ }
+
+ @Override
+ public void makeAvailable() throws IOException {
+ if (!finalLogFile.exists()) {
+ HardLink.createHardLink(compactedLogFile, finalLogFile);
+ }
+ }
+ @Override
+ public void finalizeAndCleanup() {
+ if (compactedLogFile.exists()) {
+ if (!compactedLogFile.delete()) {
+ LOG.warn("Could not delete file: " + compactedLogFile);
+ }
+ }
+ if (compactingLogFile.exists()) {
+ if (!compactingLogFile.delete()) {
+ LOG.warn("Could not delete file: " + compactingLogFile);
+ }
+ }
}
- synchronized boolean isFlushedEntryLog(Long entryLogId) {
- return entryLogsStatusMap.containsKey(entryLogId) && entryLogsStatusMap.get(entryLogId)
- || entryLogId < leastUnflushedLogId;
+ @Override
+ public long getDstLogId() {
+ return compactionLogId;
+ }
+ @Override
+ public long getSrcLogId() {
+ return logIdToCompact;
}
+
+ @Override
+ public String toString() {
+ return MoreObjects.toStringHelper(this)
+ .add("logId", compactionLogId)
+ .add("compactedLogId", logIdToCompact)
+ .add("compactingLogFile", compactingLogFile)
+ .add("compactedLogFile", compactedLogFile)
+ .add("finalLogFile", finalLogFile)
+ .toString();
+ }
+ }
+
+ @Override
+ public Collection incompleteCompactionLogs() {
+ List ledgerDirs = ledgerDirsManager.getAllLedgerDirs();
+ List compactionLogs = new ArrayList<>();
+
+ for (File dir : ledgerDirs) {
+ File[] compactingPhaseFiles = dir.listFiles(
+ file -> file.getName().endsWith(TransactionalEntryLogCompactor.COMPACTING_SUFFIX));
+ if (compactingPhaseFiles != null) {
+ for (File file : compactingPhaseFiles) {
+ if (file.delete()) {
+ LOG.info("Deleted failed compaction file {}", file);
+ }
+ }
+ }
+ File[] compactedPhaseFiles = dir.listFiles(
+ file -> file.getName().endsWith(TransactionalEntryLogCompactor.COMPACTED_SUFFIX));
+ if (compactedPhaseFiles != null) {
+ for (File compactedFile : compactedPhaseFiles) {
+ LOG.info("Found compacted log file {} has partially flushed index, recovering index.",
+ compactedFile);
+
+ File compactingLogFile = new File("/doesntexist");
+ long compactionLogId = -1L;
+ long compactedLogId = -1L;
+ String[] parts = compactedFile.getName().split(Pattern.quote("."));
+ boolean valid = true;
+ if (parts.length != 4) {
+ valid = false;
+ } else {
+ try {
+ compactionLogId = Long.parseLong(parts[0], 16);
+ compactedLogId = Long.parseLong(parts[2], 16);
+ } catch (NumberFormatException nfe) {
+ valid = false;
+ }
+ }
+
+ if (!valid) {
+ LOG.info("Invalid compacted file found ({}), deleting", compactedFile);
+ if (!compactedFile.delete()) {
+ LOG.warn("Couldn't delete invalid compacted file ({})", compactedFile);
+ }
+ continue;
+ }
+ File finalLogFile = new File(compactedFile.getParentFile(), compactionLogId + ".log");
+
+ compactionLogs.add(
+ new EntryLoggerCompactionEntryLog(compactionLogId, compactedLogId,
+ compactingLogFile, compactedFile, finalLogFile));
+ }
+ }
+ }
+ return compactionLogs;
+ }
+
+ private static File compactedLogFileFromCompacting(File compactionLogFile, long compactingLogId) {
+ File dir = compactionLogFile.getParentFile();
+ String filename = compactionLogFile.getName();
+ String newSuffix = ".log." + EntryLogger.logId2HexString(compactingLogId)
+ + TransactionalEntryLogCompactor.COMPACTED_SUFFIX;
+ String hardLinkFilename = filename.replace(TransactionalEntryLogCompactor.COMPACTING_SUFFIX, newSuffix);
+ return new File(dir, hardLinkFilename);
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
index 600d0f41494..35d401b9426 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
@@ -33,15 +33,14 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-
import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Supplier;
import lombok.Getter;
import org.apache.bookkeeper.bookie.BookieException.EntryLogMetadataMapException;
import org.apache.bookkeeper.bookie.GarbageCollector.GarbageCleaner;
import org.apache.bookkeeper.bookie.stats.GarbageCollectorStats;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.bookie.storage.ldb.PersistentEntryLogMetadataMap;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.meta.LedgerManager;
@@ -89,7 +88,7 @@ public class GarbageCollectorThread extends SafeRunnable {
final boolean isForceGCAllowWhenNoSpace;
// Entry Logger Handle
- final EntryLogger entryLogger;
+ final EntryLoggerIface entryLogger;
final AbstractLogCompactor compactor;
// Stats loggers for garbage collection operations
@@ -110,9 +109,6 @@ public class GarbageCollectorThread extends SafeRunnable {
volatile boolean running = true;
- // track the last scanned successfully log id
- long scannedLogId = 0;
-
// Boolean to trigger a forced GC.
final AtomicBoolean forceGarbageCollection = new AtomicBoolean(false);
// Boolean to disable major compaction, when disk is almost full
@@ -138,8 +134,9 @@ public class GarbageCollectorThread extends SafeRunnable {
public GarbageCollectorThread(ServerConfiguration conf, LedgerManager ledgerManager,
final LedgerDirsManager ledgerDirsManager,
final CompactableLedgerStorage ledgerStorage,
+ EntryLoggerIface entryLogger,
StatsLogger statsLogger) throws IOException {
- this(conf, ledgerManager, ledgerDirsManager, ledgerStorage, statsLogger,
+ this(conf, ledgerManager, ledgerDirsManager, ledgerStorage, entryLogger, statsLogger,
Executors.newSingleThreadScheduledExecutor(new DefaultThreadFactory("GarbageCollectorThread")));
}
@@ -154,6 +151,7 @@ public GarbageCollectorThread(ServerConfiguration conf,
LedgerManager ledgerManager,
final LedgerDirsManager ledgerDirsManager,
final CompactableLedgerStorage ledgerStorage,
+ EntryLoggerIface entryLogger,
StatsLogger statsLogger,
ScheduledExecutorService gcExecutor)
throws IOException {
@@ -161,7 +159,7 @@ public GarbageCollectorThread(ServerConfiguration conf,
this.conf = conf;
this.ledgerDirsManager = ledgerDirsManager;
- this.entryLogger = ledgerStorage.getEntryLogger();
+ this.entryLogger = entryLogger;
this.entryLogMetaMap = createEntryLogMetadataMap();
this.ledgerStorage = ledgerStorage;
this.gcWaitTime = conf.getGcWaitTime();
@@ -667,15 +665,7 @@ protected void compactEntryLog(EntryLogMetadata entryLogMeta) {
* @throws EntryLogMetadataMapException
*/
protected void extractMetaFromEntryLogs() throws EntryLogMetadataMapException {
- // Entry Log ID's are just a long value that starts at 0 and increments by 1 when the log fills up and we roll
- // to a new one. We scan entry logs as follows:
- // - entryLogPerLedgerEnabled is false: Extract it for every entry log except for the current one (un-flushed).
- // - entryLogPerLedgerEnabled is true: Scan all flushed entry logs up to the highest known id.
- Supplier finalEntryLog = () -> conf.isEntryLogPerLedgerEnabled() ? entryLogger.getLastLogId() :
- entryLogger.getLeastUnflushedLogId();
- boolean hasExceptionWhenScan = false;
- boolean increaseScannedLogId = true;
- for (long entryLogId = scannedLogId; entryLogId < finalEntryLog.get(); entryLogId++) {
+ for (long entryLogId : entryLogger.getFlushedLogIds()) {
// Comb the current entry log file if it has not already been extracted.
if (entryLogMetaMap.containsKey(entryLogId)) {
continue;
@@ -687,15 +677,6 @@ protected void extractMetaFromEntryLogs() throws EntryLogMetadataMapException {
continue;
}
- // If entryLogPerLedgerEnabled is true, we will look for entry log files beyond getLeastUnflushedLogId()
- // that have been explicitly rotated or below getLeastUnflushedLogId().
- if (conf.isEntryLogPerLedgerEnabled() && !entryLogger.isFlushedEntryLog(entryLogId)) {
- LOG.info("Entry log {} not flushed (entryLogPerLedgerEnabled). Starting next iteration at this point.",
- entryLogId);
- increaseScannedLogId = false;
- continue;
- }
-
LOG.info("Extracting entry log meta from entryLogId: {}", entryLogId);
try {
@@ -711,18 +692,10 @@ protected void extractMetaFromEntryLogs() throws EntryLogMetadataMapException {
entryLogMetaMap.put(entryLogId, entryLogMeta);
}
} catch (IOException e) {
- hasExceptionWhenScan = true;
LOG.warn("Premature exception when processing " + entryLogId
+ " recovery will take care of the problem", e);
}
- // if scan failed on some entry log, we don't move 'scannedLogId' to next id
- // if scan succeed, we don't need to scan it again during next gc run,
- // we move 'scannedLogId' to next id (unless entryLogPerLedgerEnabled is true
- // and we have found and un-flushed entry log already).
- if (!hasExceptionWhenScan && (!conf.isEntryLogPerLedgerEnabled() || increaseScannedLogId)) {
- ++scannedLogId;
- }
}
}
@@ -746,4 +719,4 @@ public GarbageCollectionStatus getGarbageCollectionStatus() {
.minorCompactionCounter(gcStats.getMinorCompactionCounter().get())
.build();
}
-}
\ No newline at end of file
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InMemoryEntryLogMetadataMap.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InMemoryEntryLogMetadataMap.java
index 88648428e00..b8bdfe14f1d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InMemoryEntryLogMetadataMap.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InMemoryEntryLogMetadataMap.java
@@ -57,6 +57,16 @@ public int size() {
return entryLogMetaMap.size();
}
+ @Override
+ public boolean isEmpty() {
+ return entryLogMetaMap.isEmpty();
+ }
+
+ @Override
+ public void clear() {
+ entryLogMetaMap.clear();
+ }
+
@Override
public void close() throws IOException {
entryLogMetaMap.clear();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
index 5d0b1ef37be..07b4d543d4b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
@@ -195,7 +195,7 @@ public void initializeWithEntryLogger(ServerConfiguration conf,
ledgerCache = new LedgerCacheImpl(conf, activeLedgers,
null == indexDirsManager ? ledgerDirsManager : indexDirsManager, statsLogger);
gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager,
- this, statsLogger.scope("gc"));
+ this, entryLogger, statsLogger.scope("gc"));
pageSize = conf.getPageSize();
ledgerDirsManager.addLedgerDirsListener(getLedgerDirsListener());
// Expose Stats
@@ -286,7 +286,7 @@ public void shutdown() throws InterruptedException {
LOG.info("Shutting down GC thread");
gcThread.shutdown();
LOG.info("Shutting down entry logger");
- entryLogger.shutdown();
+ entryLogger.close();
try {
ledgerCache.close();
} catch (IOException e) {
@@ -513,7 +513,6 @@ public void flushEntriesLocationsIndex() throws IOException {
ledgerCache.flushLedger(true);
}
- @Override
public EntryLogger getEntryLogger() {
return entryLogger;
}
@@ -537,7 +536,7 @@ protected void processEntry(long ledgerId, long entryId, ByteBuf entry, boolean
/*
* Log the entry
*/
- long pos = entryLogger.addEntry(ledgerId, entry, rollLog);
+ long pos = entryLogger.addEntry(ledgerId, entry);
/*
* Set offset of entry id to be the current ledger position
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
index ad80d4ac1af..432109d7aee 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedStorageRegenerateIndexOp.java
@@ -30,7 +30,7 @@
import java.util.Set;
import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
import org.apache.bookkeeper.common.util.Watcher;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.proto.checksum.DigestManager;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java
index 2a683dcefa5..cac379e08a1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/ReadOnlyEntryLogger.java
@@ -36,7 +36,7 @@ public ReadOnlyEntryLogger(ServerConfiguration conf) throws IOException {
}
@Override
- protected boolean removeEntryLog(long entryLogId) {
+ public boolean removeEntryLog(long entryLogId) {
// can't remove entry log in readonly mode
return false;
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
index 305cdc0a718..83b87029e2f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
@@ -260,7 +260,7 @@ public void cancelWaitForLastAddConfirmedUpdate(long ledgerId,
@Override
public void checkpoint(final Checkpoint checkpoint) throws IOException {
long numBytesFlushed = memTable.flush(this, checkpoint);
- interleavedLedgerStorage.getEntryLogger().prepareSortedLedgerStorageCheckpoint(numBytesFlushed);
+ ((EntryLogger) interleavedLedgerStorage.getEntryLogger()).prepareSortedLedgerStorageCheckpoint(numBytesFlushed);
interleavedLedgerStorage.checkpoint(checkpoint);
}
@@ -315,9 +315,9 @@ public void onSizeLimitReached(final Checkpoint cp) throws IOException {
public void run() {
try {
LOG.info("Started flushing mem table.");
- interleavedLedgerStorage.getEntryLogger().prepareEntryMemTableFlush();
+ ((EntryLogger) interleavedLedgerStorage.getEntryLogger()).prepareEntryMemTableFlush();
memTable.flush(SortedLedgerStorage.this);
- if (interleavedLedgerStorage.getEntryLogger().commitEntryMemTableFlush()) {
+ if (((EntryLogger) interleavedLedgerStorage.getEntryLogger()).commitEntryMemTableFlush()) {
interleavedLedgerStorage.checkpointer.startCheckpoint(cp);
}
} catch (Exception e) {
@@ -340,7 +340,6 @@ BookieStateManager getStateManager(){
return (BookieStateManager) stateManager;
}
- @Override
public EntryLogger getEntryLogger() {
return interleavedLedgerStorage.getEntryLogger();
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java
index aec934fa159..4fc270c76e8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/TransactionalEntryLogCompactor.java
@@ -23,14 +23,15 @@
import io.netty.buffer.ByteBuf;
-import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
-import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
+import org.apache.bookkeeper.bookie.storage.CompactionEntryLog;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
+
import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.util.HardLink;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,18 +46,18 @@ public class TransactionalEntryLogCompactor extends AbstractLogCompactor {
private static final Logger LOG = LoggerFactory.getLogger(TransactionalEntryLogCompactor.class);
- final EntryLogger entryLogger;
+ final EntryLoggerIface entryLogger;
final CompactableLedgerStorage ledgerStorage;
final List offsets = new ArrayList<>();
// compaction log file suffix
- static final String COMPACTING_SUFFIX = ".log.compacting";
+ public static final String COMPACTING_SUFFIX = ".log.compacting";
// flushed compaction log file suffix
- static final String COMPACTED_SUFFIX = ".compacted";
+ public static final String COMPACTED_SUFFIX = ".compacted";
public TransactionalEntryLogCompactor(
ServerConfiguration conf,
- EntryLogger entryLogger,
+ EntryLoggerIface entryLogger,
CompactableLedgerStorage ledgerStorage,
LogRemovalListener logRemover) {
super(conf, logRemover);
@@ -70,25 +71,10 @@ public TransactionalEntryLogCompactor(
@Override
public void cleanUpAndRecover() {
// clean up compacting logs and recover index for already compacted logs
- List ledgerDirs = entryLogger.getLedgerDirsManager().getAllLedgerDirs();
- for (File dir : ledgerDirs) {
- File[] compactingPhaseFiles = dir.listFiles(file -> file.getName().endsWith(COMPACTING_SUFFIX));
- if (compactingPhaseFiles != null) {
- for (File file : compactingPhaseFiles) {
- if (file.delete()) {
- LOG.info("Deleted failed compaction file {}", file);
- }
- }
- }
- File[] compactedPhaseFiles = dir.listFiles(file -> file.getName().endsWith(COMPACTED_SUFFIX));
- if (compactedPhaseFiles != null) {
- for (File compactedFile : compactedPhaseFiles) {
- LOG.info("Found compacted log file {} has partially flushed index, recovering index.",
- compactedFile);
- CompactionPhase updateIndex = new UpdateIndexPhase(compactedFile, true);
- updateIndex.run();
- }
- }
+ for (CompactionEntryLog log : entryLogger.incompleteCompactionLogs()) {
+ LOG.info("Found compacted log file {} has partially flushed index, recovering index.", log);
+ CompactionPhase updateIndex = new UpdateIndexPhase(log, true);
+ updateIndex.run();
}
}
@@ -97,19 +83,26 @@ public boolean compact(EntryLogMetadata metadata) {
if (metadata != null) {
LOG.info("Compacting entry log {} with usage {}.",
metadata.getEntryLogId(), metadata.getUsage());
- CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata);
+ CompactionEntryLog compactionLog;
+ try {
+ compactionLog = entryLogger.newCompactionLog(metadata.getEntryLogId());
+ } catch (IOException ioe) {
+ LOG.error("Exception creating new compaction entry log", ioe);
+ return false;
+ }
+ CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata, compactionLog);
if (!scanEntryLog.run()) {
LOG.info("Compaction for entry log {} end in ScanEntryLogPhase.", metadata.getEntryLogId());
return false;
}
- File compactionLogFile = entryLogger.getCurCompactionLogFile();
- CompactionPhase flushCompactionLog = new FlushCompactionLogPhase(metadata.getEntryLogId());
+
+ CompactionPhase flushCompactionLog = new FlushCompactionLogPhase(compactionLog);
if (!flushCompactionLog.run()) {
LOG.info("Compaction for entry log {} end in FlushCompactionLogPhase.", metadata.getEntryLogId());
return false;
}
- File compactedLogFile = getCompactedLogFile(compactionLogFile, metadata.getEntryLogId());
- CompactionPhase updateIndex = new UpdateIndexPhase(compactedLogFile);
+
+ CompactionPhase updateIndex = new UpdateIndexPhase(compactionLog);
if (!updateIndex.run()) {
LOG.info("Compaction for entry log {} end in UpdateIndexPhase.", metadata.getEntryLogId());
return false;
@@ -161,16 +154,17 @@ boolean run() {
*/
class ScanEntryLogPhase extends CompactionPhase {
private final EntryLogMetadata metadata;
+ private final CompactionEntryLog compactionLog;
- ScanEntryLogPhase(EntryLogMetadata metadata) {
+ ScanEntryLogPhase(EntryLogMetadata metadata, CompactionEntryLog compactionLog) {
super("ScanEntryLogPhase");
this.metadata = metadata;
+ this.compactionLog = compactionLog;
}
@Override
void start() throws IOException {
// scan entry log into compaction log and offset list
- entryLogger.createNewCompactionLog();
entryLogger.scanEntryLog(metadata.getEntryLogId(), new EntryLogScanner() {
@Override
public boolean accept(long ledgerId) {
@@ -189,7 +183,7 @@ public void process(long ledgerId, long offset, ByteBuf entry) throws IOExceptio
ledgerId, lid, entryId, offset);
throw new IOException("Invalid entry found @ offset " + offset);
}
- long newOffset = entryLogger.addEntryForCompaction(ledgerId, entry);
+ long newOffset = compactionLog.addEntry(ledgerId, entry);
offsets.add(new EntryLocation(ledgerId, entryId, newOffset));
if (LOG.isDebugEnabled()) {
@@ -207,7 +201,7 @@ boolean complete() {
// no valid entries is compacted, delete entry log file
LOG.info("No valid entry is found in entry log after scan, removing entry log now.");
logRemovalListener.removeEntryLog(metadata.getEntryLogId());
- entryLogger.removeCurCompactionLog();
+ compactionLog.abort();
return false;
}
return true;
@@ -217,9 +211,8 @@ boolean complete() {
void abort() {
offsets.clear();
// since we haven't flushed yet, we only need to delete the unflushed compaction file.
- entryLogger.removeCurCompactionLog();
+ compactionLog.abort();
}
-
}
/**
@@ -230,51 +223,35 @@ void abort() {
* a hardlink file "3.log.1.compacted" should be created, and "3.log.compacting" should be deleted.
*/
class FlushCompactionLogPhase extends CompactionPhase {
- private final long compactingLogId;
- private File compactedLogFile;
+ final CompactionEntryLog compactionLog;
- FlushCompactionLogPhase(long compactingLogId) {
+ FlushCompactionLogPhase(CompactionEntryLog compactionLog) {
super("FlushCompactionLogPhase");
- this.compactingLogId = compactingLogId;
+ this.compactionLog = compactionLog;
}
@Override
void start() throws IOException {
// flush the current compaction log.
- File compactionLogFile = entryLogger.getCurCompactionLogFile();
- if (compactionLogFile == null || !compactionLogFile.exists()) {
- throw new IOException("Compaction log doesn't exist during flushing");
- }
- entryLogger.flushCompactionLog();
+ compactionLog.flush();
}
@Override
boolean complete() throws IOException {
- // create a hard link file named "x.log.y.compacted" for file "x.log.compacting".
- // where x is compactionLogId and y is compactingLogId.
- File compactionLogFile = entryLogger.getCurCompactionLogFile();
- if (compactionLogFile == null || !compactionLogFile.exists()) {
- LOG.warn("Compaction log doesn't exist any more after flush");
+ try {
+ compactionLog.markCompacted();
+ return true;
+ } catch (IOException ioe) {
+ LOG.warn("Error marking compaction as done", ioe);
return false;
}
- compactedLogFile = getCompactedLogFile(compactionLogFile, compactingLogId);
- if (compactedLogFile != null && !compactedLogFile.exists()) {
- HardLink.createHardLink(compactionLogFile, compactedLogFile);
- }
- entryLogger.removeCurCompactionLog();
- return true;
}
@Override
void abort() {
offsets.clear();
// remove compaction log file and its hardlink
- entryLogger.removeCurCompactionLog();
- if (compactedLogFile != null && compactedLogFile.exists()) {
- if (!compactedLogFile.delete()) {
- LOG.warn("Could not delete compacted log file {}", compactedLogFile);
- }
- }
+ compactionLog.abort();
}
}
@@ -289,41 +266,29 @@ void abort() {
*
This phase can also used to recover partially flushed index when we pass isInRecovery=true
*/
class UpdateIndexPhase extends CompactionPhase {
- File compactedLogFile;
- File newEntryLogFile;
+ final CompactionEntryLog compactionLog;
private final boolean isInRecovery;
- public UpdateIndexPhase(File compactedLogFile) {
- this(compactedLogFile, false);
+ public UpdateIndexPhase(CompactionEntryLog compactionLog) {
+ this(compactionLog, false);
}
- public UpdateIndexPhase(File compactedLogFile, boolean isInRecovery) {
+ public UpdateIndexPhase(CompactionEntryLog compactionLog, boolean isInRecovery) {
super("UpdateIndexPhase");
- this.compactedLogFile = compactedLogFile;
+ this.compactionLog = compactionLog;
this.isInRecovery = isInRecovery;
}
@Override
void start() throws IOException {
- if (compactedLogFile != null && compactedLogFile.exists()) {
- File dir = compactedLogFile.getParentFile();
- String compactedFilename = compactedLogFile.getName();
- // create a hard link "x.log" for file "x.log.y.compacted"
- this.newEntryLogFile = new File(dir, compactedFilename.substring(0,
- compactedFilename.indexOf(".log") + 4));
- if (!newEntryLogFile.exists()) {
- HardLink.createHardLink(compactedLogFile, newEntryLogFile);
- }
- if (isInRecovery) {
- recoverEntryLocations(EntryLogger.fileName2LogId(newEntryLogFile.getName()));
- }
- if (!offsets.isEmpty()) {
- // update entry locations and flush index
- ledgerStorage.updateEntriesLocations(offsets);
- ledgerStorage.flushEntriesLocationsIndex();
- }
- } else {
- throw new IOException("Failed to find compacted log file in UpdateIndexPhase");
+ compactionLog.makeAvailable();
+ if (isInRecovery) {
+ recoverEntryLocations(compactionLog);
+ }
+ if (!offsets.isEmpty()) {
+ // update entry locations and flush index
+ ledgerStorage.updateEntriesLocations(offsets);
+ ledgerStorage.flushEntriesLocationsIndex();
}
}
@@ -332,16 +297,8 @@ boolean complete() {
// When index is flushed, and entry log is removed,
// delete the ".compacted" file to indicate this phase is completed.
offsets.clear();
- if (compactedLogFile != null) {
- if (!compactedLogFile.delete()) {
- LOG.warn("Could not delete compacted log file {}", compactedLogFile);
- }
- // Now delete the old entry log file since it's compacted
- String compactedFilename = compactedLogFile.getName();
- String oldEntryLogFilename = compactedFilename.substring(compactedFilename.indexOf(".log") + 5);
- long entryLogId = EntryLogger.fileName2LogId(oldEntryLogFilename);
- logRemovalListener.removeEntryLog(entryLogId);
- }
+ compactionLog.finalizeAndCleanup();
+ logRemovalListener.removeEntryLog(compactionLog.getSrcLogId());
return true;
}
@@ -353,8 +310,8 @@ void abort() {
/**
* Scan entry log to recover entry locations.
*/
- private void recoverEntryLocations(long compactedLogId) throws IOException {
- entryLogger.scanEntryLog(compactedLogId, new EntryLogScanner() {
+ private void recoverEntryLocations(CompactionEntryLog compactionLog) throws IOException {
+ compactionLog.scan(new EntryLogScanner() {
@Override
public boolean accept(long ledgerId) {
return true;
@@ -370,23 +327,11 @@ public void process(long ledgerId, long offset, ByteBuf entry) throws IOExceptio
ledgerId, lid, entryId, offset);
throw new IOException("Invalid entry found @ offset " + offset);
}
- long location = (compactedLogId << 32L) | (offset + 4);
+ long location = (compactionLog.getDstLogId() << 32L) | (offset + 4);
offsets.add(new EntryLocation(lid, entryId, location));
}
});
- LOG.info("Recovered {} entry locations from compacted log {}", offsets.size(), compactedLogId);
+ LOG.info("Recovered {} entry locations from compacted log {}", offsets.size(), compactionLog.getDstLogId());
}
}
-
- File getCompactedLogFile(File compactionLogFile, long compactingLogId) {
- if (compactionLogFile == null) {
- return null;
- }
- File dir = compactionLogFile.getParentFile();
- String filename = compactionLogFile.getName();
- String newSuffix = ".log." + EntryLogger.logId2HexString(compactingLogId) + COMPACTED_SUFFIX;
- String hardLinkFilename = filename.replace(COMPACTING_SUFFIX, newSuffix);
- return new File(dir, hardLinkFilename);
- }
-
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/CompactionEntryLog.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/CompactionEntryLog.java
new file mode 100644
index 00000000000..fcbfcf702f2
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/CompactionEntryLog.java
@@ -0,0 +1,90 @@
+/**
+ *
+ * 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 io.netty.buffer.ByteBuf;
+import java.io.IOException;
+
+/**
+ * An entrylog to received compacted entries.
+ *
+ * The expected lifecycle for a compaction entry log is:
+ * 1. Creation
+ * 2. Mark compacted
+ * 3. Make available
+ * 4. Cleanup
+ *
+ * Abort can happen at during any step.
+ */
+public interface CompactionEntryLog {
+ /**
+ * Add an entry to the log.
+ * @param ledgerId the ledger the entry belong to
+ * @param entry the payload of the entry
+ * @return the position to which the entry was written
+ */
+ long addEntry(long ledgerId, ByteBuf entry) throws IOException;
+
+ /**
+ * Scan the entry log, reading out all contained entries.
+ */
+ void scan(EntryLogScanner scanner) throws IOException;
+
+ /**
+ * Flush any unwritten entries to physical storage.
+ */
+ void flush() throws IOException;
+
+ /**
+ * Abort the compaction log. This should delete any resources held
+ * by this log.
+ */
+ void abort();
+
+ /**
+ * Mark the compaction log as compacted.
+ * From this point, the heavy work of copying entries from one log
+ * to another should be done. We don't want to repeat that work,
+ * so this method should take steps to ensure that if the bookie crashes
+ * we can resume the compaction from this point.
+ */
+ void markCompacted() throws IOException;
+
+ /**
+ * Make the log written by the compaction process available for reads.
+ */
+ void makeAvailable() throws IOException;
+
+ /**
+ * Clean up any temporary resources that were used by the compaction process.
+ */
+ void finalizeAndCleanup();
+
+ /**
+ * Get the log ID of the entrylog to which compacted entries are being written.
+ */
+ long getDstLogId();
+
+ /**
+ * Get the log ID of the entrylog which is being compacted.
+ */
+ long getSrcLogId();
+}
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/EntryLogScanner.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogScanner.java
new file mode 100644
index 00000000000..bcca2b19889
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLogScanner.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.IOException;
+
+/**
+ * Scan entries in a entry log file.
+ */
+public interface EntryLogScanner {
+ /**
+ * Tests whether or not the entries belongs to the specified ledger
+ * should be processed.
+ *
+ * @param ledgerId
+ * Ledger ID.
+ * @return true if and only the entries of the ledger should be scanned.
+ */
+ boolean accept(long ledgerId);
+
+ /**
+ * Process an entry.
+ *
+ * @param ledgerId
+ * Ledger ID.
+ * @param offset
+ * File offset of this entry.
+ * @param entry
+ * Entry ByteBuf
+ * @throws IOException
+ */
+ void process(long ledgerId, long offset, ByteBuf entry) throws IOException;
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLoggerIface.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLoggerIface.java
new file mode 100644
index 00000000000..d1e3282acbc
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/EntryLoggerIface.java
@@ -0,0 +1,128 @@
+/**
+ *
+ * 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 io.netty.buffer.ByteBuf;
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.bookkeeper.bookie.AbstractLogCompactor;
+import org.apache.bookkeeper.bookie.Bookie.NoEntryException;
+import org.apache.bookkeeper.bookie.EntryLogMetadata;
+
+
+/**
+ * Entry logger. Sequentially writes entries for a large number of ledgers to
+ * a small number of log files, to avoid many random writes.
+ * When an entry is added, a location is returned, which consists of the ID of the
+ * log into which the entry was added, and the offset of that entry within the log.
+ * The location is a long, with 32 bits each for the log ID and the offset. This
+ * naturally limits the offset and thus the size of the log to Integer.MAX_VALUE.
+ */
+public interface EntryLoggerIface extends AutoCloseable {
+ /**
+ * Add an entry for ledger ```ledgerId``` to the entrylog.
+ * @param ledgerId the ledger for which the entry is being added
+ * @param buf the contents of the entry (this method does not take ownership of the refcount)
+ * @return the location in the entry log of the added entry
+ */
+ long addEntry(long ledgerId, ByteBuf buf) throws IOException;
+
+ /**
+ * Read an entry from an entrylog location.
+ * @param entryLocation the location from which to read the entry
+ * @return the entry
+ */
+ ByteBuf readEntry(long entryLocation)
+ throws IOException, NoEntryException;
+ /**
+ * Read an entry from an entrylog location, and verify that is matches the
+ * expected ledger and entry ID.
+ * @param ledgerId the ledgerID to match
+ * @param entryId the entryID to match
+ * @param entryLocation the location from which to read the entry
+ * @return the entry
+ */
+ ByteBuf readEntry(long ledgerId, long entryId, long entryLocation)
+ throws IOException, NoEntryException;
+
+ /**
+ * Flush any outstanding writes to disk.
+ */
+ void flush() throws IOException;
+
+ @Override
+ void close() throws IOException;
+
+ /**
+ * Create a new entrylog into which compacted entries can be added.
+ * There is a 1-1 mapping between logs thats that are being compacted
+ * and the log the compacted entries are written to.
+ */
+ CompactionEntryLog newCompactionLog(long logToCompact) throws IOException;
+
+ /**
+ * Return a collection of all the compaction entry logs which have been
+ * compacted, but have not been cleaned up.
+ */
+ Collection incompleteCompactionLogs();
+
+ /**
+ * Get the log ids for the set of logs which have been completely flushed to
+ * disk.
+ * Only log ids in this set are considered for either compaction or garbage
+ * collection.
+ */
+ Collection getFlushedLogIds();
+
+ /**
+ * Scan the given entrylog, returning all entries contained therein.
+ */
+ void scanEntryLog(long entryLogId, EntryLogScanner scanner) throws IOException;
+
+ /**
+ * Retrieve metadata for the given entrylog ID.
+ * The metadata contains the size of the log, the size of the data in the log which is still
+ * active, and a list of all the ledgers contained in the log and the size of the data stored
+ * for each ledger.
+ */
+ default EntryLogMetadata getEntryLogMetadata(long entryLogId) throws IOException {
+ return getEntryLogMetadata(entryLogId, null);
+ }
+
+ /**
+ * Retrieve metadata for the given entrylog ID.
+ * The metadata contains the size of the log, the size of the data in the log which is still
+ * active, and a list of all the ledgers contained in the log and the size of the data stored
+ * for each ledger.
+ */
+ EntryLogMetadata getEntryLogMetadata(long entryLogId, AbstractLogCompactor.Throttler throttler) throws IOException;
+
+ /**
+ * Check whether an entrylog with the given ID exists.
+ */
+ boolean logExists(long logId);
+
+ /**
+ * Delete the entrylog with the given ID.
+ * @return false if the entrylog doesn't exist.
+ */
+ boolean removeEntryLog(long entryLogId);
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Buffer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Buffer.java
new file mode 100644
index 00000000000..03a9342f085
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Buffer.java
@@ -0,0 +1,270 @@
+/**
+ *
+ * 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.common.util.ExceptionMessageHelper.exMsg;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.ReferenceCountUtil;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+
+import org.apache.bookkeeper.common.util.nativeio.NativeIO;
+
+/**
+ * A utility buffer class to be used with native calls.
+ *
+ * Buffers are page aligned (4k pages).
+ *
+ * The wrapper mostly handles writes between ByteBuffers and
+ * ByteBufs. It also provides a method for padding the buffer to the next
+ * alignment, so writes can have an aligned size also (as required by
+ * direct I/O). The padding is done with 0xF0, so that if it is read as
+ * an integer, or long, the value will be negative (assuming the read is
+ * a java read, and thus an signed int).
+ */
+class Buffer {
+ /* Padding byte must have MSB set, so if read at the start
+ * of an integer or long, the returned value is negative. */
+ public static final byte PADDING_BYTE = (byte) 0xF0;
+
+ /* Some machines can live with 512 alignment, but others
+ * appear to require 4096, so go with 4096, which is page
+ * alignment */
+ public static final int ALIGNMENT = 4096;
+ private static final int MAX_ALIGNMENT = Integer.MAX_VALUE & ~(ALIGNMENT - 1);
+ static final byte[] PADDING = generatePadding();
+
+ final NativeIO nativeIO;
+ final int bufferSize;
+ ByteBuf buffer;
+ ByteBuffer byteBuffer;
+ long pointer = 0;
+
+ Buffer(NativeIO nativeIO, int bufferSize) throws IOException {
+ checkArgument(isAligned(bufferSize),
+ "Buffer size not aligned %d", bufferSize);
+
+ this.buffer = allocateAligned(ALIGNMENT, bufferSize);
+
+ this.nativeIO = nativeIO;
+ this.bufferSize = bufferSize;
+ byteBuffer = buffer.nioBuffer(0, bufferSize);
+ byteBuffer.order(ByteOrder.BIG_ENDIAN);
+ }
+
+ private ByteBuf allocateAligned(int alignment, int bufferSize) {
+ ByteBuf buf = PooledByteBufAllocator.DEFAULT.directBuffer(bufferSize + alignment);
+ long addr = buf.memoryAddress();
+ if ((addr & (alignment - 1)) == 0) {
+ // The address is already aligned
+ pointer = addr;
+ return buf.slice(0, bufferSize);
+ } else {
+ int alignOffset = (int) (alignment - (addr & (alignment - 1)));
+ pointer = addr + alignOffset;
+ return buf.slice(alignOffset, bufferSize);
+ }
+ }
+
+ /**
+ * @return whether there is space in the buffer for size bytes.
+ */
+ boolean hasSpace(int size) throws IOException {
+ if (size > bufferSize) {
+ throw new IOException(exMsg("Write too large").kv("writeSize", size)
+ .kv("maxSize", bufferSize).toString());
+ }
+ return byteBuffer.remaining() >= size;
+ }
+
+ /**
+ * @return whether the buffer can honour a read of size at offset.
+ */
+ boolean hasData(int offset, int size) {
+ return offset + size <= bufferSize;
+ }
+
+ /**
+ * Write an integer to buffer. Progresses the position of the buffer by 4 bytes.
+ */
+ void writeInt(int value) throws IOException {
+ byteBuffer.putInt(value);
+ }
+
+ /**
+ * Write a bytebuf to this buffer. Progresses the position of the buffer by the
+ * number of readable bytes of the bytebuf. Progresses the readerIndex of the passed
+ * bytebuf by the number of bytes read (i.e. to the end).
+ */
+ void writeByteBuf(ByteBuf bytebuf) throws IOException {
+ int bytesWritten = bytebuf.readableBytes();
+ ByteBuffer bytesToPut = bytebuf.nioBuffer();
+ byteBuffer.put(bytesToPut);
+ bytebuf.skipBytes(bytesWritten);
+ }
+
+ /**
+ * Read an integer from the buffer at the given offset. The offset is in bytes.
+ */
+ int readInt(int offset) throws IOException {
+ if (!hasData(offset, Integer.BYTES)) {
+ throw new IOException(exMsg("Buffer cannot satify int read")
+ .kv("offset", offset)
+ .kv("bufferSize", bufferSize).toString());
+ }
+ try {
+ return byteBuffer.getInt(offset);
+ } catch (Exception e) {
+ throw new IOException(exMsg("Error reading int")
+ .kv("byteBuffer", byteBuffer.toString())
+ .kv("offset", offset)
+ .kv("bufferSize", bufferSize).toString(), e);
+ }
+ }
+
+ /**
+ * Read a long from the buffer at the given offset. The offset is in bytes.
+ */
+ long readLong(int offset) throws IOException {
+ if (!hasData(offset, Long.BYTES)) {
+ throw new IOException(exMsg("Buffer cannot satify long read")
+ .kv("offset", offset)
+ .kv("bufferSize", bufferSize).toString());
+ }
+ try {
+ return byteBuffer.getLong(offset);
+ } catch (Exception e) {
+ throw new IOException(exMsg("Error reading long")
+ .kv("byteBuffer", byteBuffer.toString())
+ .kv("offset", offset)
+ .kv("bufferSize", bufferSize).toString(), e);
+ }
+ }
+
+ /**
+ * Read a bytebuf of size from the buffer at the given offset.
+ * If there are not enough bytes in the buffer to satify the read, some of the bytes are read
+ * into the byte buffer and the number of bytes read is returned.
+ */
+ int readByteBuf(ByteBuf buffer, int offset, int size) throws IOException {
+ int originalLimit = byteBuffer.limit();
+ byteBuffer.position(offset);
+ int bytesToRead = Math.min(size, byteBuffer.capacity() - offset);
+ byteBuffer.limit(offset + bytesToRead);
+ try {
+ buffer.writeBytes(byteBuffer);
+ } catch (Exception e) {
+ throw new IOException(exMsg("Error reading buffer")
+ .kv("byteBuffer", byteBuffer.toString())
+ .kv("offset", offset).kv("size", size)
+ .kv("bufferSize", bufferSize).toString(), e);
+ } finally {
+ byteBuffer.limit(originalLimit);
+ }
+ return bytesToRead;
+ }
+
+ /**
+ * The data pointer object for the native buffer. This can be used
+ * by JNI method which take a char* or void*.
+ */
+ long pointer() {
+ return pointer;
+ }
+
+ long pointer(long offset, long expectedWrite) {
+ if (offset == 0) {
+ return pointer;
+ } else {
+ if (offset + expectedWrite > byteBuffer.capacity()) {
+ throw new IllegalArgumentException(
+ exMsg("Buffer overflow").kv("offset", offset).kv("expectedWrite", expectedWrite)
+ .kv("capacity", byteBuffer.capacity()).toString());
+ }
+
+ return pointer + offset;
+ }
+ }
+ /**
+ * @return the number of bytes which have been written to this buffer.
+ */
+ int position() {
+ return byteBuffer.position();
+ }
+
+ /**
+ * @return the size of the buffer (i.e. the max number of bytes writable, or the max offset readable)
+ */
+ int size() {
+ return bufferSize;
+ }
+
+ /**
+ * Pad the buffer to the next alignment position.
+ * @return the position of the next alignment. This should be used as the size argument to make aligned writes.
+ */
+ int padToAlignment() {
+ int bufferPos = byteBuffer.position();
+ int nextAlignment = nextAlignment(bufferPos);
+ byteBuffer.put(PADDING, 0, nextAlignment - bufferPos);
+ return nextAlignment;
+ }
+
+ /**
+ * Clear the bytes written. This doesn't actually destroy the data, but moves the position back to the start of
+ * the buffer.
+ */
+ void reset() {
+ byteBuffer.clear();
+ }
+
+ /**
+ * Free the memory that backs this buffer.
+ */
+ void free() {
+ ReferenceCountUtil.safeRelease(buffer);
+ buffer = null;
+ byteBuffer = null;
+ }
+ private static byte[] generatePadding() {
+ byte[] padding = new byte[ALIGNMENT];
+ Arrays.fill(padding, (byte) PADDING_BYTE);
+ return padding;
+ }
+
+ static boolean isAligned(long size) {
+ return size >= 0 && ((ALIGNMENT - 1) & size) == 0;
+ }
+
+ static int nextAlignment(int pos) {
+ checkArgument(pos <= MAX_ALIGNMENT,
+ "position (0x%x) must be lower or equal to max alignment (0x%x)",
+ pos, MAX_ALIGNMENT);
+ checkArgument(pos >= 0, "position (0x%x) must be positive", pos);
+ return (pos + (ALIGNMENT - 1)) & ~(ALIGNMENT - 1);
+ }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferPool.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferPool.java
new file mode 100644
index 00000000000..901f9ee1e72
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/BufferPool.java
@@ -0,0 +1,69 @@
+/**
+ *
+ * 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 java.io.IOException;
+import java.util.concurrent.ArrayBlockingQueue;
+
+import org.apache.bookkeeper.common.util.nativeio.NativeIO;
+
+/**
+ * BufferPool.
+ */
+public class BufferPool implements AutoCloseable {
+ private final int maxPoolSize;
+ private final ArrayBlockingQueue pool;
+ BufferPool(NativeIO nativeIO, int bufferSize, int maxPoolSize) throws IOException {
+ this.maxPoolSize = maxPoolSize;
+ pool = new ArrayBlockingQueue<>(maxPoolSize);
+ for (int i = 0; i < maxPoolSize; i++) {
+ pool.add(new Buffer(nativeIO, bufferSize));
+ }
+ }
+
+ Buffer acquire() throws IOException {
+ try {
+ return pool.take();
+ } catch (InterruptedException ie) {
+ Thread.currentThread().interrupt();
+ throw new IOException(ie);
+ }
+ }
+
+ void release(Buffer buffer) {
+ buffer.reset();
+ if (!pool.add(buffer)) {
+ buffer.free();
+ }
+ }
+
+ @Override
+ public void close() {
+ while (true) {
+ Buffer b = pool.poll();
+ if (b == null) {
+ break;
+ }
+
+ b.free();
+ }
+ }
+}
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..700fee8c3de
--- /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.EntryLoggerIface;
+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 EntryLoggerIface {
+ 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..aef11ae6cb8
--- /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.EntryLoggerIface}) 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.
+ *
+ *
+ */
+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:
+ *
+ *
+ */
+ 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/directentrylogger/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/package-info.java
new file mode 100644
index 00000000000..a714867782b
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Support for bookie entry logs using Direct IO.
+ */
+package org.apache.bookkeeper.bookie.storage.directentrylogger;
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 0a563a1c9ce..3a1acf4d09f 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.EntryLogger;
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.EntryLoggerIface;
+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));
+
+ EntryLoggerIface 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 EntryLogger(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,
+ EntryLoggerIface 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
@@ -427,6 +510,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/LedgersIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
index 6e481ca5695..643bf5b7488 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
@@ -38,9 +38,9 @@
import java.util.Set;
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.EntryLogger;
-import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
import org.apache.bookkeeper.bookie.Journal;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.util.BookKeeperConstants;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
index 55c5c90c31e..9fa704104a6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
@@ -35,8 +35,8 @@
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.EntryLogger;
-import org.apache.bookkeeper.bookie.EntryLogger.EntryLogScanner;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.stats.NullStatsLogger;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
index ec289fd16c2..35c53fe620a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
@@ -196,6 +196,21 @@ public int size() throws EntryLogMetadataMapException {
}
}
+ @Override
+ public void clear() throws EntryLogMetadataMapException {
+ try {
+ try (KeyValueStorage.Batch b = metadataMapDB.newBatch();
+ CloseableIterator itr = metadataMapDB.keys()) {
+ while (itr.hasNext()) {
+ b.remove(itr.next());
+ }
+ b.flush();
+ }
+ } catch (IOException e) {
+ throw new EntryLogMetadataMapException(e);
+ }
+ }
+
@Override
public void close() throws IOException {
if (isClosed.compareAndSet(false, true)) {
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 2697abde4b7..78f31bea9d2 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
@@ -59,7 +59,6 @@
import org.apache.bookkeeper.bookie.Checkpointer;
import org.apache.bookkeeper.bookie.CompactableLedgerStorage;
import org.apache.bookkeeper.bookie.EntryLocation;
-import org.apache.bookkeeper.bookie.EntryLogger;
import org.apache.bookkeeper.bookie.GarbageCollectionStatus;
import org.apache.bookkeeper.bookie.GarbageCollectorThread;
import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification;
@@ -68,6 +67,7 @@
import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener;
import org.apache.bookkeeper.bookie.LedgerEntryPage;
import org.apache.bookkeeper.bookie.StateManager;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch;
import org.apache.bookkeeper.common.util.Watcher;
@@ -91,7 +91,7 @@
*
This is meant only to be used from {@link DbLedgerStorage}.
*/
public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage {
- private final EntryLogger entryLogger;
+ private final EntryLoggerIface entryLogger;
private final LedgerMetadataIndex ledgerIndex;
private final EntryLocationIndex entryLocationIndex;
@@ -145,7 +145,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,
+ EntryLoggerIface entryLogger, StatsLogger statsLogger,
ByteBufAllocator allocator, ScheduledExecutorService gcExecutor, long writeCacheSize, long readCacheSize,
int readAheadCacheBatchSize) throws IOException {
checkArgument(ledgerDirsManager.getAllLedgerDirs().size() == 1,
@@ -185,8 +186,9 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le
TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES,
TransientLedgerInfo.LEDGER_INFO_CACHING_TIME_MINUTES, TimeUnit.MINUTES);
- entryLogger = new EntryLogger(conf, ledgerDirsManager, null, statsLogger, allocator);
- gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, statsLogger);
+ this.entryLogger = entryLogger;
+ gcThread = new GarbageCollectorThread(conf, ledgerManager, ledgerDirsManager, this, entryLogger,
+ statsLogger);
dbLedgerStorageStats = new DbLedgerStorageStats(
ledgerDirStatsLogger,
@@ -260,7 +262,7 @@ public void shutdown() throws InterruptedException {
flush();
gcThread.shutdown();
- entryLogger.shutdown();
+ entryLogger.close();
cleanupExecutor.shutdown();
cleanupExecutor.awaitTermination(1, TimeUnit.SECONDS);
@@ -584,8 +586,8 @@ private void fillReadAheadCache(long orginalLedgerId, long firstEntryId, long fi
while (count < readAheadCacheBatchSize
&& size < maxReadAheadBytesSize
&& currentEntryLogId == firstEntryLogId) {
- ByteBuf entry = entryLogger.internalReadEntry(orginalLedgerId, firstEntryId, currentEntryLocation,
- false /* validateEntry */);
+ ByteBuf entry = entryLogger.readEntry(orginalLedgerId,
+ firstEntryId, currentEntryLocation);
try {
long currentEntryLedgerId = entry.getLong(0);
@@ -717,7 +719,7 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
Batch batch = entryLocationIndex.newBatch();
writeCacheBeingFlushed.forEach((ledgerId, entryId, entry) -> {
try {
- long location = entryLogger.addEntry(ledgerId, entry, true);
+ long location = entryLogger.addEntry(ledgerId, entry);
entryLocationIndex.addLocation(batch, ledgerId, entryId, location);
} catch (IOException e) {
throw new RuntimeException(e);
@@ -858,8 +860,8 @@ public void updateEntriesLocations(Iterable locations) throws IOE
entryLocationIndex.updateLocations(locations);
}
- @Override
- public EntryLogger getEntryLogger() {
+ @VisibleForTesting
+ EntryLoggerIface getEntryLogger() {
return entryLogger;
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/package-info.java
new file mode 100644
index 00000000000..f8744532700
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+/**
+ * Provides a Bookie server that stores entries for clients.
+ */
+package org.apache.bookkeeper.bookie.storage;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java
index 12e419f97e0..c8ca28f3f0b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/tools/cli/commands/bookie/ReadLogCommand.java
@@ -27,6 +27,7 @@
import lombok.experimental.Accessors;
import org.apache.bookkeeper.bookie.EntryLogger;
import org.apache.bookkeeper.bookie.ReadOnlyEntryLogger;
+import org.apache.bookkeeper.bookie.storage.EntryLogScanner;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.tools.cli.helpers.BookieCommand;
import org.apache.bookkeeper.tools.framework.CliFlags;
@@ -175,7 +176,7 @@ private void scanEntryLogForPositionRange(ServerConfiguration conf, long logId,
LOG.info("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for PositionRange: "
+ rangeStartPos + " - " + rangeEndPos);
final MutableBoolean entryFound = new MutableBoolean(false);
- scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+ scanEntryLog(conf, logId, new EntryLogScanner() {
private MutableBoolean stopScanning = new MutableBoolean(false);
@Override
@@ -222,7 +223,7 @@ public void process(long ledgerId, long entryStartPos, ByteBuf entry) throws IOE
* @param logId Entry Log Id
* @param scanner Entry Log Scanner
*/
- private void scanEntryLog(ServerConfiguration conf, long logId, EntryLogger.EntryLogScanner scanner)
+ private void scanEntryLog(ServerConfiguration conf, long logId, EntryLogScanner scanner)
throws IOException {
initEntryLogger(conf);
entryLogger.scanEntryLog(logId, scanner);
@@ -250,7 +251,7 @@ private void scanEntryLogForSpecificEntry(ServerConfiguration conf, long logId,
LOG.info("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)" + " for LedgerId "
+ ledgerId + ((entryId == -1) ? "" : " for EntryId " + entryId));
final MutableBoolean entryFound = new MutableBoolean(false);
- scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+ scanEntryLog(conf, logId, new EntryLogScanner() {
@Override
public boolean accept(long candidateLedgerId) {
return ((candidateLedgerId == ledgerId) && ((!entryFound.booleanValue()) || (entryId == -1)));
@@ -284,7 +285,7 @@ public void process(long candidateLedgerId, long startPos, ByteBuf entry) {
*/
private void scanEntryLog(ServerConfiguration conf, long logId, final boolean printMsg) throws Exception {
LOG.info("Scan entry log " + logId + " (" + Long.toHexString(logId) + ".log)");
- scanEntryLog(conf, logId, new EntryLogger.EntryLogScanner() {
+ scanEntryLog(conf, logId, new EntryLogScanner() {
@Override
public boolean accept(long ledgerId) {
return true;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentLongLongHashMap.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentLongLongHashMap.java
index ec1d232ab1b..6db4e32ad7c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentLongLongHashMap.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/ConcurrentLongLongHashMap.java
@@ -891,4 +891,20 @@ private static void checkBiggerEqualZero(long n) {
throw new IllegalArgumentException("Keys and values must be >= 0");
}
}
+
+ @Override
+ public String toString() {
+ StringBuilder sb = new StringBuilder();
+ sb.append("ConcurrentLongLongHashMap{");
+
+ int headerLen = sb.length();
+ forEach((k, v) -> {
+ sb.append(k).append(" => ").append(v).append(", ");
+ });
+ if (sb.length() > headerLen) {
+ sb.setLength(sb.length() - 2);
+ }
+ sb.append("}");
+ return sb.toString();
+ }
}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
index 47726b0b0c5..1751a84d12b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
@@ -35,6 +35,7 @@
import static org.junit.Assert.fail;
import com.google.common.util.concurrent.UncheckedExecutionException;
+
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import io.netty.buffer.UnpooledByteBufAllocator;
@@ -44,10 +45,8 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.Enumeration;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
-import java.util.Map;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
@@ -57,6 +56,7 @@
import org.apache.bookkeeper.bookie.BookieException.EntryLogMetadataMapException;
import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
+import org.apache.bookkeeper.bookie.storage.CompactionEntryLog;
import org.apache.bookkeeper.bookie.storage.ldb.PersistentEntryLogMetadataMap;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -74,13 +74,14 @@
import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
import org.apache.bookkeeper.test.TestStatsProvider;
import org.apache.bookkeeper.util.DiskChecker;
-import org.apache.bookkeeper.util.HardLink;
import org.apache.bookkeeper.util.TestUtils;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
import org.apache.zookeeper.AsyncCallback;
+
import org.junit.Before;
import org.junit.Test;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -651,7 +652,7 @@ public void testMinorCompactionWithEntryLogPerLedgerEnabled() throws Exception {
bkc.deleteLedger(lhs[2].getId());
// Need to wait until entry log 3 gets flushed before initiating GC to satisfy assertions.
- while (!getGCThread().entryLogger.isFlushedEntryLog(3L)) {
+ while (!getGCThread().entryLogger.getFlushedLogIds().contains(3L)) {
TimeUnit.MILLISECONDS.sleep(100);
}
@@ -674,7 +675,7 @@ public void testMinorCompactionWithEntryLogPerLedgerEnabled() throws Exception {
// Now, let's mark E1 as flushed, as its ledger L1 has been deleted already. In this case, the GC algorithm
// should consider it for deletion.
- getGCThread().entryLogger.recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(1L);
+ ((EntryLogger) getGCThread().entryLogger).recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(1L);
getGCThread().triggerGC(true, false, false).get();
assertTrue("Found entry log file 1.log that should have been compacted in ledgerDirectory: "
+ tmpDirs.getDirs().get(0), TestUtils.hasNoneLogFiles(tmpDirs.getDirs().get(0), 1));
@@ -684,7 +685,7 @@ public void testMinorCompactionWithEntryLogPerLedgerEnabled() throws Exception {
getGCThread().triggerGC(true, false, false).get();
assertTrue("Found entry log file 0.log that should not have been compacted in ledgerDirectory: "
+ tmpDirs.getDirs().get(0), TestUtils.hasAllLogFiles(tmpDirs.getDirs().get(0), 0));
- getGCThread().entryLogger.recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(0L);
+ ((EntryLogger) getGCThread().entryLogger).recentlyCreatedEntryLogsStatus.flushRotatedEntryLog(0L);
getGCThread().triggerGC(true, false, false).get();
assertTrue("Found entry log file 0.log that should have been compacted in ledgerDirectory: "
+ tmpDirs.getDirs().get(0), TestUtils.hasNoneLogFiles(tmpDirs.getDirs().get(0), 0));
@@ -1437,80 +1438,6 @@ public void checkpointComplete(Checkpoint checkpoint,
storage.gcThread.doCompactEntryLogs(threshold, limit);
}
- /**
- * Test extractMetaFromEntryLogs optimized method to avoid excess memory usage.
- */
- public void testExtractMetaFromEntryLogs() throws Exception {
- // restart bookies
- restartBookies(c -> {
- // Always run this test with Throttle enabled.
- c.setIsThrottleByBytes(true);
- return c;
- });
- ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
- File tmpDir = tmpDirs.createNew("bkTest", ".dir");
- File curDir = BookieImpl.getCurrentDirectory(tmpDir);
- BookieImpl.checkDirectoryStructure(curDir);
- conf.setLedgerDirNames(new String[] { tmpDir.toString() });
-
- LedgerDirsManager dirs = new LedgerDirsManager(conf, conf.getLedgerDirs(),
- new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()));
- final Set ledgers = Collections
- .newSetFromMap(new ConcurrentHashMap());
-
- LedgerManager manager = getLedgerManager(ledgers);
-
- CheckpointSource checkpointSource = new CheckpointSource() {
-
- @Override
- public Checkpoint newCheckpoint() {
- return null;
- }
-
- @Override
- public void checkpointComplete(Checkpoint checkpoint,
- boolean compact) throws IOException {
- }
- };
- InterleavedLedgerStorage storage = new InterleavedLedgerStorage();
- storage.initialize(conf, manager, dirs, dirs,
- NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
- storage.setCheckpointSource(checkpointSource);
- storage.setCheckpointer(Checkpointer.NULL);
-
-
- for (long ledger = 0; ledger <= 10; ledger++) {
- ledgers.add(ledger);
- for (int entry = 1; entry <= 50; entry++) {
- try {
- storage.addEntry(genEntry(ledger, entry, ENTRY_SIZE));
- } catch (IOException e) {
- //ignore exception on failure to add entry.
- }
- }
- }
-
- storage.flush();
- storage.shutdown();
-
- storage = new InterleavedLedgerStorage();
- storage.initialize(conf, manager, dirs, dirs, NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
- storage.setCheckpointSource(checkpointSource);
- storage.setCheckpointer(Checkpointer.NULL);
-
- long startingEntriesCount = storage.gcThread.entryLogger.getLeastUnflushedLogId()
- - storage.gcThread.scannedLogId;
- LOG.info("The old Log Entry count is: " + startingEntriesCount);
-
- Map entryLogMetaData = new HashMap<>();
- long finalEntriesCount = storage.gcThread.entryLogger.getLeastUnflushedLogId()
- - storage.gcThread.scannedLogId;
- LOG.info("The latest Log Entry count is: " + finalEntriesCount);
-
- assertTrue("The GC did not clean up entries...", startingEntriesCount != finalEntriesCount);
- assertTrue("Entries Count is zero", finalEntriesCount == 0);
- }
-
private ByteBuf genEntry(long ledger, long entry, int size) {
ByteBuf bb = Unpooled.buffer(size);
bb.writeLong(ledger);
@@ -1813,30 +1740,30 @@ public MockTransactionalEntryLogCompactor(GarbageCollectorThread gcThread) {
super(gcThread.conf,
gcThread.entryLogger,
gcThread.ledgerStorage,
- (long entry) -> {
- try {
- gcThread.removeEntryLog(entry);
- } catch (EntryLogMetadataMapException e) {
- LOG.warn("Failed to remove entry-log metadata {}", entry, e);
- }
- });
+ (long entry) -> {
+ try {
+ gcThread.removeEntryLog(entry);
+ } catch (EntryLogMetadataMapException e) {
+ LOG.warn("Failed to remove entry-log metadata {}", entry, e);
+ }
+ });
}
- synchronized void compactWithIndexFlushFailure(EntryLogMetadata metadata) {
+ synchronized void compactWithIndexFlushFailure(EntryLogMetadata metadata) throws IOException {
LOG.info("Compacting entry log {}.", metadata.getEntryLogId());
- CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata);
+ CompactionEntryLog compactionLog = entryLogger.newCompactionLog(metadata.getEntryLogId());
+
+ CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata, compactionLog);
if (!scanEntryLog.run()) {
LOG.info("Compaction for {} end in ScanEntryLogPhase.", metadata.getEntryLogId());
return;
}
- File compactionLogFile = entryLogger.getCurCompactionLogFile();
- CompactionPhase flushCompactionLog = new FlushCompactionLogPhase(metadata.getEntryLogId());
+ CompactionPhase flushCompactionLog = new FlushCompactionLogPhase(compactionLog);
if (!flushCompactionLog.run()) {
LOG.info("Compaction for {} end in FlushCompactionLogPhase.", metadata.getEntryLogId());
return;
}
- File compactedLogFile = getCompactedLogFile(compactionLogFile, metadata.getEntryLogId());
- CompactionPhase partialFlushIndexPhase = new PartialFlushIndexPhase(compactedLogFile);
+ CompactionPhase partialFlushIndexPhase = new PartialFlushIndexPhase(compactionLog);
if (!partialFlushIndexPhase.run()) {
LOG.info("Compaction for {} end in PartialFlushIndexPhase.", metadata.getEntryLogId());
return;
@@ -1845,21 +1772,21 @@ synchronized void compactWithIndexFlushFailure(EntryLogMetadata metadata) {
LOG.info("Compacted entry log : {}.", metadata.getEntryLogId());
}
- synchronized void compactWithLogFlushFailure(EntryLogMetadata metadata) {
+ synchronized void compactWithLogFlushFailure(EntryLogMetadata metadata) throws IOException {
LOG.info("Compacting entry log {}", metadata.getEntryLogId());
- CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata);
+ CompactionEntryLog compactionLog = entryLogger.newCompactionLog(metadata.getEntryLogId());
+
+ CompactionPhase scanEntryLog = new ScanEntryLogPhase(metadata, compactionLog);
if (!scanEntryLog.run()) {
LOG.info("Compaction for {} end in ScanEntryLogPhase.", metadata.getEntryLogId());
return;
}
- File compactionLogFile = entryLogger.getCurCompactionLogFile();
- CompactionPhase logFlushFailurePhase = new LogFlushFailurePhase(metadata.getEntryLogId());
+ CompactionPhase logFlushFailurePhase = new LogFlushFailurePhase(compactionLog);
if (!logFlushFailurePhase.run()) {
LOG.info("Compaction for {} end in FlushCompactionLogPhase.", metadata.getEntryLogId());
return;
}
- File compactedLogFile = getCompactedLogFile(compactionLogFile, metadata.getEntryLogId());
- CompactionPhase updateIndex = new UpdateIndexPhase(compactedLogFile);
+ CompactionPhase updateIndex = new UpdateIndexPhase(compactionLog);
if (!updateIndex.run()) {
LOG.info("Compaction for entry log {} end in UpdateIndexPhase.", metadata.getEntryLogId());
return;
@@ -1870,45 +1797,35 @@ synchronized void compactWithLogFlushFailure(EntryLogMetadata metadata) {
private class PartialFlushIndexPhase extends UpdateIndexPhase {
- public PartialFlushIndexPhase(File compactedLogFile) {
- super(compactedLogFile);
+ public PartialFlushIndexPhase(CompactionEntryLog compactionLog) {
+ super(compactionLog);
}
@Override
void start() throws IOException {
- if (compactedLogFile != null && compactedLogFile.exists()) {
- File dir = compactedLogFile.getParentFile();
- String compactedFilename = compactedLogFile.getName();
- // create a hard link "x.log" for file "x.log.y.compacted"
- this.newEntryLogFile = new File(dir, compactedFilename.substring(0,
- compactedFilename.indexOf(".log") + 4));
- File hardlinkFile = new File(dir, newEntryLogFile.getName());
- if (!hardlinkFile.exists()) {
- HardLink.createHardLink(compactedLogFile, hardlinkFile);
- }
- assertTrue(offsets.size() > 1);
- // only flush index for one entry location
- EntryLocation el = offsets.get(0);
- ledgerStorage.updateEntriesLocations(offsets);
- ledgerStorage.flushEntriesLocationsIndex();
- throw new IOException("Flush ledger index encounter exception");
- }
+ compactionLog.makeAvailable();
+ assertTrue(offsets.size() > 1);
+ // only flush index for one entry location
+ EntryLocation el = offsets.get(0);
+ ledgerStorage.updateEntriesLocations(offsets);
+ ledgerStorage.flushEntriesLocationsIndex();
+ throw new IOException("Flush ledger index encounter exception");
}
}
private class LogFlushFailurePhase extends FlushCompactionLogPhase {
- LogFlushFailurePhase(long compactingLogId) {
- super(compactingLogId);
+ LogFlushFailurePhase(CompactionEntryLog compactionEntryLog) {
+ super(compactionEntryLog);
}
@Override
void start() throws IOException {
// flush the current compaction log
- entryLogger.flushCompactionLog();
+ compactionLog.flush();
throw new IOException("Encounter IOException when trying to flush compaction log");
}
}
}
-}
\ No newline at end of file
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
index 215e15942da..6cfee84838a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CreateNewLogTest.java
@@ -516,7 +516,7 @@ public void testCreateNewLogAndCompactionLog() throws Exception {
if (i % 2 == 0) {
((EntryLogManagerBase) el.getEntryLogManager()).createNewLog((long) i);
} else {
- el.createNewCompactionLog();
+ el.newCompactionLog(i);
}
} catch (IOException e) {
LOG.error("Received exception while creating newLog", e);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
index e3627f40abc..f1a8831fd9d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
@@ -114,7 +114,7 @@ public void setUp() throws Exception {
@After
public void tearDown() throws Exception {
if (null != this.entryLogger) {
- entryLogger.shutdown();
+ entryLogger.close();
}
for (File dir : tempDirs) {
@@ -125,7 +125,7 @@ public void tearDown() throws Exception {
@Test
public void testDeferCreateNewLog() throws Exception {
- entryLogger.shutdown();
+ entryLogger.close();
// mark `curDir` as filled
this.conf.setMinUsableSizeForEntryLogCreation(1);
@@ -149,7 +149,7 @@ public void testDeferCreateNewLog() throws Exception {
@Test
public void testDeferCreateNewLogWithoutEnoughDiskSpaces() throws Exception {
- entryLogger.shutdown();
+ entryLogger.close();
// mark `curDir` as filled
this.conf.setMinUsableSizeForEntryLogCreation(Long.MAX_VALUE);
@@ -182,7 +182,7 @@ public void testCorruptEntryLog() throws Exception {
entryLogger.addEntry(3L, generateEntry(3, 1).nioBuffer());
entryLogger.addEntry(2L, generateEntry(2, 1).nioBuffer());
entryLogger.flush();
- entryLogger.shutdown();
+ entryLogger.close();
// now lets truncate the file to corrupt the last entry, which simulates a partial write
File f = new File(curDir, "0.log");
RandomAccessFile raf = new RandomAccessFile(f, "rw");
@@ -235,7 +235,7 @@ public void testMissingLogId() throws Exception {
positions[i][j] = logger.addEntry((long) i, generateEntry(i, j).nioBuffer());
}
logger.flush();
- logger.shutdown();
+ logger.close();
}
// delete last log id
File lastLogId = new File(curDir, "lastId");
@@ -250,7 +250,7 @@ public void testMissingLogId() throws Exception {
positions[i][j] = logger.addEntry((long) i, generateEntry(i, j).nioBuffer());
}
logger.flush();
- logger.shutdown();
+ logger.close();
}
EntryLogger newLogger = new EntryLogger(conf, dirsMgr);
@@ -291,7 +291,7 @@ public void testEntryLoggerShouldThrowFNFEIfDirectoriesDoesNotExist()
.getLocalizedMessage());
} finally {
if (entryLogger != null) {
- entryLogger.shutdown();
+ entryLogger.close();
}
}
}
@@ -367,7 +367,7 @@ public void testRecoverFromLedgersMapOnV0EntryLog() throws Exception {
entryLogger.addEntry(2L, generateEntry(2, 1).nioBuffer());
entryLogger.addEntry(1L, generateEntry(1, 2).nioBuffer());
((EntryLogManagerBase) entryLogger.getEntryLogManager()).createNewLog(EntryLogger.UNASSIGNED_LEDGERID);
- entryLogger.shutdown();
+ entryLogger.close();
// Rewrite the entry log header to be on V0 format
File f = new File(curDir, "0.log");
@@ -404,7 +404,7 @@ public void testRecoverFromLedgersMapOnV0EntryLog() throws Exception {
*/
@Test
public void testPreAllocateLog() throws Exception {
- entryLogger.shutdown();
+ entryLogger.close();
// enable pre-allocation case
conf.setEntryLogFilePreAllocationEnabled(true);
@@ -417,7 +417,7 @@ public void testPreAllocateLog() throws Exception {
entryLogger.addEntry(1L, generateEntry(1, 1).nioBuffer());
// the Future is not null all the time
assertNotNull(entryLogger.getEntryLoggerAllocator().getPreallocationFuture());
- entryLogger.shutdown();
+ entryLogger.close();
// disable pre-allocation case
conf.setEntryLogFilePreAllocationEnabled(false);
@@ -462,7 +462,7 @@ public void testGetEntryLogsSet() throws Exception {
*/
@Test
public void testFlushOrder() throws Exception {
- entryLogger.shutdown();
+ entryLogger.close();
int logSizeLimit = 256 * 1024;
conf.setEntryLogPerLedgerEnabled(false);
@@ -512,7 +512,7 @@ public void run() {
* here we are adding entry of size logSizeLimit with
* rolllog=true, so it would create a new entrylog.
*/
- entryLogger.addEntry(123, generateEntry(123, 456, logSizeLimit), true);
+ entryLogger.addEntry(123, generateEntry(123, 456, logSizeLimit));
} catch (InterruptedException | BrokenBarrierException | IOException e) {
LOG.error("Exception happened for entryLogManager.createNewLog", e);
exceptionHappened.set(true);
@@ -1087,7 +1087,7 @@ public void testEntryLogManagerExpiryRemoval() throws Exception {
*/
@Test
public void testCacheMaximumSizeEvictionPolicy() throws Exception {
- entryLogger.shutdown();
+ entryLogger.close();
final int cacheMaximumSize = 20;
ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
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 f070143cb4b..38d04b9018a 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
@@ -20,18 +20,39 @@
*/
package org.apache.bookkeeper.bookie;
+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.containsInAnyOrder;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
import static org.mockito.Mockito.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.when;
import static org.mockito.MockitoAnnotations.initMocks;
+import java.io.File;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.conf.TestBKConfiguration;
import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.MockLedgerManager;
+import org.apache.bookkeeper.slogger.Slogger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.test.TmpDirs;
+
+import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@@ -39,15 +60,16 @@
import org.mockito.Mock;
import org.mockito.Spy;
import org.powermock.reflect.Whitebox;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Unit test for {@link GarbageCollectorThread}.
*/
@SuppressWarnings("deprecation")
public class GarbageCollectorThreadTest {
- private static final Logger LOG = LoggerFactory.getLogger(GarbageCollectorThreadTest.class);
+ private static final Slogger slog = Slogger.CONSOLE;
+
+ private final TmpDirs tmpDirs = new TmpDirs();
+
@InjectMocks
@Spy
private GarbageCollectorThread mockGCThread;
@@ -64,10 +86,15 @@ public class GarbageCollectorThreadTest {
@Before
public void setUp() throws Exception {
- when(ledgerStorage.getEntryLogger()).thenReturn(mock(EntryLogger.class));
+ conf.setAllowLoopback(true);
initMocks(this);
}
+ @After
+ public void cleanup() throws Exception {
+ tmpDirs.cleanup();
+ }
+
@Test
public void testCompactEntryLogWithException() throws Exception {
AbstractLogCompactor mockCompactor = mock(AbstractLogCompactor.class);
@@ -88,25 +115,129 @@ public void testCalculateUsageBucket() {
// Valid range for usage is [0.0 to 1.0]
final int numBuckets = 10;
int[] usageBuckets = new int[numBuckets];
+ String[] bucketNames = new String[numBuckets];
for (int i = 0; i < numBuckets; i++) {
usageBuckets[i] = 0;
+ bucketNames[i] = String.format("%d%%", (i + 1) * 10);
}
int items = 10000;
+
for (int item = 0; item <= items; item++) {
double usage = ((double) item / (double) items);
int index = mockGCThread.calculateUsageIndex(numBuckets, usage);
- Assert.assertFalse("Boundary condition exceeded", index < 0 || index >= numBuckets);
- LOG.debug("Mapped {} usage to {}}\n", usage, index);
+ assertFalse("Boundary condition exceeded", index < 0 || index >= numBuckets);
+ slog.kv("usage", usage)
+ .kv("index", index)
+ .info("Mapped usage to index");
usageBuckets[index]++;
}
- LOG.info(
- "Compaction: entry log usage buckets[10% 20% 30% 40% 50% 60% 70% 80% 90% 100%] = {}",
- usageBuckets);
+
+ Slogger sl = slog.ctx();
+ for (int i = 0; i < numBuckets; i++) {
+ sl = sl.kv(bucketNames[i], usageBuckets[i]);
+ }
+ sl.info("Compaction: entry log usage buckets");
+
int sum = 0;
for (int i = 0; i < numBuckets; i++) {
sum += usageBuckets[i];
}
Assert.assertEquals("Incorrect number of items", items + 1, sum);
}
+
+ @Test
+ public void testExtractMetaFromEntryLogsLegacy() throws Exception {
+ File ledgerDir = tmpDirs.createNew("testExtractMeta", "ledgers");
+ testExtractMetaFromEntryLogs(
+ 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(EntryLoggerIface entryLogger, File ledgerDir)
+ throws Exception {
+
+ MockLedgerStorage storage = new MockLedgerStorage();
+ MockLedgerManager lm = new MockLedgerManager();
+
+ GarbageCollectorThread gcThread = new GarbageCollectorThread(
+ TestBKConfiguration.newServerConfiguration(), lm,
+ newDirsManager(ledgerDir),
+ storage, entryLogger,
+ NullStatsLogger.INSTANCE);
+
+ // 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);
+ entryLogger.flush();
+
+ storage.setMasterKey(1L, new byte[0]);
+ storage.setMasterKey(2L, new byte[0]);
+ storage.setMasterKey(3L, new byte[0]);
+
+ assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2));
+ assertTrue(entryLogger.logExists(logId3));
+
+ // all ledgers exist, nothing should disappear
+ final EntryLogMetadataMap entryLogMetaMap = gcThread.getEntryLogMetaMap();
+ gcThread.extractMetaFromEntryLogs();
+
+ assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1, logId2));
+ assertTrue(entryLogMetaMap.containsKey(logId1));
+ assertTrue(entryLogMetaMap.containsKey(logId2));
+ assertTrue(entryLogger.logExists(logId3));
+
+ // log 2 is 100% ledger 2, so it should disappear if ledger 2 is deleted
+ entryLogMetaMap.clear();
+ storage.deleteLedger(2L);
+ gcThread.extractMetaFromEntryLogs();
+
+ assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId1));
+ assertTrue(entryLogMetaMap.containsKey(logId1));
+ assertTrue(entryLogger.logExists(logId3));
+
+ // delete all ledgers, all logs except the current should be deleted
+ entryLogMetaMap.clear();
+ storage.deleteLedger(1L);
+ storage.deleteLedger(3L);
+ gcThread.extractMetaFromEntryLogs();
+
+ assertThat(entryLogger.getFlushedLogIds(), empty());
+ assertTrue(entryLogMetaMap.isEmpty());
+ assertTrue(entryLogger.logExists(logId3));
+
+ // add enough entries to roll log, log 3 can not be GC'd
+ long loc6 = entryLogger.addEntry(3L, makeEntry(3L, 1L, 25000));
+ assertThat(logIdFromLocation(loc6), greaterThan(logIdFromLocation(loc5)));
+ entryLogger.flush();
+ assertThat(entryLogger.getFlushedLogIds(), containsInAnyOrder(logId3));
+
+ entryLogMetaMap.clear();
+ gcThread.extractMetaFromEntryLogs();
+
+ assertThat(entryLogger.getFlushedLogIds(), empty());
+ assertTrue(entryLogMetaMap.isEmpty());
+ assertFalse(entryLogger.logExists(logId3));
+ }
}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
index d62c8697c2b..5c525d1236c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
@@ -372,7 +372,8 @@ conf, new TestBookieImpl(conf),
}
handle.close();
// simulate rolling entrylog
- ((EntryLogManagerBase) ledgerStorage.getEntryLogger().getEntryLogManager()).createNewLog(ledgerId);
+ ((EntryLogManagerBase) ((EntryLogger) ledgerStorage.getEntryLogger()).getEntryLogManager())
+ .createNewLog(ledgerId);
// sleep for a bit for checkpoint to do its task
executorController.advance(Duration.ofMillis(500));
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 ea3ff04921a..941c7b42129 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;
@@ -41,7 +42,7 @@
/**
* A mock for running tests that require ledger storage.
*/
-public class MockLedgerStorage implements LedgerStorage {
+public class MockLedgerStorage implements CompactableLedgerStorage {
private static class LedgerInfo {
boolean limbo = false;
@@ -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,
@@ -242,7 +244,7 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
@Override
public void deleteLedger(long ledgerId) throws IOException {
- throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+ ledgers.remove(ledgerId);
}
@Override
@@ -262,27 +264,27 @@ public ByteBuf getExplicitLac(long ledgerId) throws IOException {
@Override
public LedgerStorage getUnderlyingLedgerStorage() {
- return LedgerStorage.super.getUnderlyingLedgerStorage();
+ return CompactableLedgerStorage.super.getUnderlyingLedgerStorage();
}
@Override
public void forceGC() {
- LedgerStorage.super.forceGC();
+ CompactableLedgerStorage.super.forceGC();
}
@Override
public List localConsistencyCheck(Optional rateLimiter) throws IOException {
- return LedgerStorage.super.localConsistencyCheck(rateLimiter);
+ return CompactableLedgerStorage.super.localConsistencyCheck(rateLimiter);
}
@Override
public boolean isInForceGC() {
- return LedgerStorage.super.isInForceGC();
+ return CompactableLedgerStorage.super.isInForceGC();
}
@Override
public List getGarbageCollectionStatus() {
- return LedgerStorage.super.getGarbageCollectionStatus();
+ return CompactableLedgerStorage.super.getGarbageCollectionStatus();
}
@Override
@@ -290,6 +292,25 @@ public PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws I
throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
}
+ @Override
+ public Iterable getActiveLedgersInRange(long firstLedgerId, long lastLedgerId)
+ throws IOException {
+ throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+ }
+
+ public List getUpdatedLocations() {
+ return entryLocations;
+ }
+
+ @Override
+ public void updateEntriesLocations(Iterable locations) throws IOException {
+ synchronized (entryLocations) {
+ for (EntryLocation l : locations) {
+ entryLocations.add(l);
+ }
+ }
+ }
+
@Override
public EnumSet getStorageStateFlags() throws IOException {
return storageStateFlags;
@@ -304,4 +325,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/SlowInterleavedLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
index bdd7af354ed..4de4f86286c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SlowInterleavedLedgerStorage.java
@@ -76,9 +76,9 @@ public void flush() throws IOException {
}
@Override
- public long addEntry(long ledger, ByteBuf entry, boolean rollLog) throws IOException {
+ public long addEntry(long ledger, ByteBuf entry) throws IOException {
delayMs(addDelay);
- return super.addEntry(ledger, entry, rollLog);
+ return super.addEntry(ledger, entry);
}
@Override
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
index a60ab7e34cd..3d186c28f7e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SortedLedgerStorageCheckpointTest.java
@@ -224,12 +224,12 @@ public void testCheckpointAfterEntryLogRotated() throws Exception {
});
// simulate entry log is rotated (due to compaction)
- EntryLogManagerForSingleEntryLog entryLogManager = (EntryLogManagerForSingleEntryLog) storage.getEntryLogger()
- .getEntryLogManager();
+ EntryLogger elogger = (EntryLogger) storage.getEntryLogger();
+ EntryLogManagerForSingleEntryLog entryLogManager =
+ (EntryLogManagerForSingleEntryLog) elogger.getEntryLogManager();
entryLogManager.createNewLog(EntryLogger.UNASSIGNED_LEDGERID);
- long leastUnflushedLogId = storage.getEntryLogger().getLeastUnflushedLogId();
+
long currentLogId = entryLogManager.getCurrentLogId();
- log.info("Least unflushed entry log : current = {}, leastUnflushed = {}", currentLogId, leastUnflushedLogId);
readyLatch.countDown();
assertNull(checkpoints.poll());
@@ -246,8 +246,8 @@ public void testCheckpointAfterEntryLogRotated() throws Exception {
assertEquals(0, storage.memTable.kvmap.size());
assertTrue(
"current log " + currentLogId + " contains entries added from memtable should be forced to disk"
- + " but least unflushed log is " + storage.getEntryLogger().getLeastUnflushedLogId(),
- storage.getEntryLogger().getLeastUnflushedLogId() > currentLogId);
+ + " but flushed logs are " + elogger.getFlushedLogIds(),
+ elogger.getFlushedLogIds().contains(currentLogId));
}
}
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
new file mode 100644
index 00000000000..519bbf472e7
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java
@@ -0,0 +1,103 @@
+/**
+ *
+ * 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.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.File;
+import java.util.Arrays;
+
+import org.apache.bookkeeper.bookie.EntryLogger;
+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;
+import org.apache.bookkeeper.util.DiskChecker;
+
+/**
+ * EntryLogTestUtils.
+ */
+public class EntryLogTestUtils {
+ private static final Slogger slog = Slogger.CONSOLE;
+
+ public static LedgerDirsManager newDirsManager(File... ledgerDir) throws Exception {
+ return new LedgerDirsManager(
+ new ServerConfiguration(), ledgerDir, new DiskChecker(0.999f, 0.999f));
+ }
+
+ public static EntryLogger newLegacyEntryLogger(int logSizeLimit, File... ledgerDir) throws Exception {
+ ServerConfiguration conf = new ServerConfiguration();
+ conf.setEntryLogSizeLimit(logSizeLimit);
+ return new EntryLogger(conf, newDirsManager(ledgerDir), null,
+ 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[] 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/TestBuffer.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestBuffer.java
new file mode 100644
index 00000000000..e35de544d45
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestBuffer.java
@@ -0,0 +1,210 @@
+/**
+ *
+ * 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.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.*;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+// CHECKSTYLE.OFF: IllegalImport
+import io.netty.util.internal.PlatformDependent;
+// CHECKSTYLE.ON: IllegalImport
+
+import java.io.IOException;
+
+import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl;
+import org.junit.Test;
+
+/**
+ * TestBuffer.
+ */
+public class TestBuffer {
+
+ @Test
+ public void testIsAligned() throws Exception {
+ assertThat(Buffer.isAligned(1234), equalTo(false));
+ assertThat(Buffer.isAligned(4096), equalTo(true));
+ assertThat(Buffer.isAligned(40960), equalTo(true));
+ assertThat(Buffer.isAligned(1 << 20), equalTo(true));
+ assertThat(Buffer.isAligned(-1), equalTo(false));
+ assertThat(Buffer.isAligned(Integer.MIN_VALUE), equalTo(false));
+ assertThat(Buffer.isAligned(Integer.MAX_VALUE), equalTo(false));
+ }
+
+ @Test
+ public void testNextAlignment() throws Exception {
+ assertThat(Buffer.nextAlignment(0), equalTo(0));
+ assertThat(Buffer.nextAlignment(1), equalTo(4096));
+ assertThat(Buffer.nextAlignment(4096), equalTo(4096));
+ assertThat(Buffer.nextAlignment(4097), equalTo(8192));
+ assertThat(Buffer.nextAlignment(0x7FFFF000), equalTo(0x7FFFF000));
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testNegativePosition() throws Exception {
+ Buffer.nextAlignment(-1);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testMaxAlignment() throws Exception {
+ Buffer.nextAlignment(Integer.MAX_VALUE);
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testCreateUnaligned() throws Exception {
+ new Buffer(new NativeIOImpl(), 1234);
+ }
+
+ @Test
+ public void testWriteInt() throws Exception {
+ int bufferSize = 1 << 20;
+ Buffer b = new Buffer(new NativeIOImpl(), bufferSize);
+ assertThat(b.hasSpace(bufferSize), equalTo(true));
+ assertThat(b.position(), equalTo(0));
+ b.writeInt(0xdeadbeef);
+
+
+ assertThat(PlatformDependent.getByte(b.pointer() + 0), equalTo((byte) 0xde));
+ assertThat(PlatformDependent.getByte(b.pointer() + 1), equalTo((byte) 0xad));
+ assertThat(PlatformDependent.getByte(b.pointer() + 2), equalTo((byte) 0xbe));
+ assertThat(PlatformDependent.getByte(b.pointer() + 3), equalTo((byte) 0xef));
+
+ assertThat(b.hasSpace(bufferSize), equalTo(false));
+ assertThat(b.position(), equalTo(Integer.BYTES));
+
+ for (int i = 0; i < 10000; i++) {
+ b.writeInt(i);
+ }
+ assertThat(b.position(), equalTo(Integer.BYTES * 10001));
+ assertThat(b.hasSpace(bufferSize - (Integer.BYTES * 10001)), equalTo(true));
+ assertThat(b.hasSpace(bufferSize - (Integer.BYTES * 10000)), equalTo(false));
+
+ assertThat(b.readInt(0), equalTo(0xdeadbeef));
+ for (int i = 0; i < 10000; i++) {
+ assertThat(b.readInt((i + 1) * Integer.BYTES), equalTo(i));
+ }
+ b.reset();
+ assertThat(b.hasSpace(bufferSize), equalTo(true));
+ assertThat(b.position(), equalTo(0));
+ }
+
+ @Test
+ public void testWriteBuffer() throws Exception {
+ ByteBuf bb = Unpooled.buffer(1021);
+ fillByteBuf(bb, 0xdeadbeef);
+ int bufferSize = 1 << 20;
+ Buffer b = new Buffer(new NativeIOImpl(), bufferSize);
+ assertThat(b.position(), equalTo(0));
+ b.writeByteBuf(bb);
+ assertThat(b.position(), equalTo(1021));
+ assertThat(bb.readableBytes(), equalTo(0));
+ bb.clear();
+ fillByteBuf(bb, 0xcafecafe);
+ b.writeByteBuf(bb);
+ assertThat(bb.readableBytes(), equalTo(0));
+ assertThat(b.position(), equalTo(2042));
+
+ bb = Unpooled.buffer(2042);
+ int ret = b.readByteBuf(bb, 0, 2042);
+ assertThat(ret, equalTo(2042));
+ for (int i = 0; i < 1020 / Integer.BYTES; i++) {
+ assertThat(bb.readInt(), equalTo(0xdeadbeef));
+ }
+ assertThat(bb.readByte(), equalTo((byte) 0xde));
+ for (int i = 0; i < 1020 / Integer.BYTES; i++) {
+ assertThat(bb.readInt(), equalTo(0xcafecafe));
+ }
+ }
+
+ @Test
+ public void testPartialRead() throws Exception {
+ ByteBuf bb = Unpooled.buffer(5000);
+
+ Buffer b = new Buffer(new NativeIOImpl(), 4096);
+ for (int i = 0; i < 4096 / Integer.BYTES; i++) {
+ b.writeInt(0xdeadbeef);
+ }
+
+ int ret = b.readByteBuf(bb, 0, 5000);
+ assertThat(ret, equalTo(4096));
+ }
+
+ @Test(expected = IOException.class)
+ public void testReadIntAtBoundary() throws Exception {
+ Buffer b = new Buffer(new NativeIOImpl(), 4096);
+
+ for (int i = 0; i < 4096 / Integer.BYTES; i++) {
+ b.writeInt(0xdeadbeef);
+ }
+ assertThat(b.hasData(4092, Integer.BYTES), equalTo(true));
+ assertThat(b.hasData(4093, Integer.BYTES), equalTo(false));
+ assertThat(b.hasData(4096, Integer.BYTES), equalTo(false));
+ b.readInt(4096 - 2);
+ }
+
+ @Test(expected = IOException.class)
+ public void testReadLongAtBoundary() throws Exception {
+ Buffer b = new Buffer(new NativeIOImpl(), 4096);
+
+ for (int i = 0; i < 4096 / Integer.BYTES; i++) {
+ b.writeInt(0xdeadbeef);
+ }
+ assertThat(b.hasData(4088, Long.BYTES), equalTo(true));
+ assertThat(b.hasData(4089, Long.BYTES), equalTo(false));
+ assertThat(b.hasData(4096, Long.BYTES), equalTo(false));
+ b.readInt(4096 - 2);
+ }
+
+ @Test
+ public void testPadToAlignment() throws Exception {
+ Buffer b = new Buffer(new NativeIOImpl(), 1 << 23);
+
+ for (int i = 0; i < 1025; i++) {
+ b.writeInt(0xdededede);
+ }
+ int writtenLength = b.padToAlignment();
+
+ assertThat(writtenLength, equalTo(8192));
+ assertThat(b.readInt(1024 * Integer.BYTES), equalTo(0xdededede));
+ for (int i = 1025 * Integer.BYTES; i < writtenLength; i += Integer.BYTES) {
+ assertThat(b.readInt(i), equalTo(0xf0f0f0f0));
+ }
+ assertThat(b.readInt(writtenLength), equalTo(0));
+ }
+
+ @Test
+ public void testFree() throws Exception {
+ Buffer b = new Buffer(new NativeIOImpl(), 1 << 23);
+ b.free(); // success if process doesn't explode
+ b.free();
+ }
+
+ static void fillByteBuf(ByteBuf bb, int value) {
+ while (bb.writableBytes() >= Integer.BYTES) {
+ bb.writeInt(value);
+ }
+ for (int i = 0; i < Integer.BYTES && bb.writableBytes() > 0; i++) {
+ byte b = (byte) (value >> (Integer.BYTES - i - 1) * 8);
+ bb.writeByte(b);
+ }
+ }
+}
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..e453da60e7b
--- /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.EntryLoggerIface;
+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 (EntryLoggerIface 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 (EntryLoggerIface 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);
+ EntryLoggerIface 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 (EntryLoggerIface 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..fa870c50a0e
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java
@@ -0,0 +1,330 @@
+/**
+ *
+ * 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.EntryLogger;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
+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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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..bf400866235
--- /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.EntryLoggerIface;
+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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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 (EntryLoggerIface 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/DbLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
index 00efee9e4be..ab6ea1d7884 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
@@ -36,10 +36,10 @@
import org.apache.bookkeeper.bookie.BookieException;
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.EntryLocation;
-import org.apache.bookkeeper.bookie.EntryLogger;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
import org.apache.bookkeeper.bookie.LedgerStorage;
import org.apache.bookkeeper.bookie.TestBookieImpl;
+import org.apache.bookkeeper.bookie.storage.EntryLoggerIface;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.conf.TestBKConfiguration;
import org.apache.bookkeeper.proto.BookieProtocol;
@@ -221,13 +221,13 @@ public void testBookieCompaction() throws Exception {
// Simulate bookie compaction
SingleDirectoryDbLedgerStorage singleDirStorage = ((DbLedgerStorage) storage).getLedgerStorageList().get(0);
- EntryLogger entryLogger = singleDirStorage.getEntryLogger();
+ EntryLoggerIface entryLogger = singleDirStorage.getEntryLogger();
// Rewrite entry-3
ByteBuf newEntry3 = Unpooled.buffer(1024);
newEntry3.writeLong(4); // ledger id
newEntry3.writeLong(3); // entry id
newEntry3.writeBytes("new-entry-3".getBytes());
- long location = entryLogger.addEntry(4L, newEntry3, false);
+ long location = entryLogger.addEntry(4L, newEntry3);
List locations = Lists.newArrayList(new EntryLocation(4, 3, location));
singleDirStorage.updateEntriesLocations(locations);
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..35f872ecbaf 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.EntryLoggerIface;
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,
+ EntryLoggerIface 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,
+ EntryLoggerIface 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);
}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
index bedb76b990b..9258a155296 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
@@ -57,7 +57,6 @@
import org.apache.bookkeeper.bookie.Checkpointer;
import org.apache.bookkeeper.bookie.CompactableLedgerStorage;
import org.apache.bookkeeper.bookie.EntryLocation;
-import org.apache.bookkeeper.bookie.EntryLogger;
import org.apache.bookkeeper.bookie.GarbageCollector;
import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -672,11 +671,6 @@ public Iterable getActiveLedgersInRange(long firstLedgerId, long lastLedge
return subBkActiveLedgers.keySet();
}
- @Override
- public EntryLogger getEntryLogger() {
- return null;
- }
-
@Override
public void updateEntriesLocations(Iterable locations) throws IOException {
}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
index 8740577dc37..c534c6b6ed8 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
@@ -40,7 +40,6 @@
import org.apache.bookkeeper.bookie.Checkpointer;
import org.apache.bookkeeper.bookie.CompactableLedgerStorage;
import org.apache.bookkeeper.bookie.EntryLocation;
-import org.apache.bookkeeper.bookie.EntryLogger;
import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
import org.apache.bookkeeper.bookie.StateManager;
@@ -264,11 +263,6 @@ public Iterable getActiveLedgersInRange(long firstLedgerId, long lastLedge
return subBkActiveLedgers.keySet();
}
- @Override
- public EntryLogger getEntryLogger() {
- return null;
- }
-
@Override
public void updateEntriesLocations(Iterable locations) throws IOException {
}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
index 2b28bfc6941..d6b8b735671 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
@@ -59,7 +59,7 @@ public interface ZooKeeperCluster {
void killCluster() throws Exception;
- void sleepCluster(final int time, final TimeUnit timeUnit, final CountDownLatch l)
+ void sleepCluster(int time, final TimeUnit timeUnit, final CountDownLatch l)
throws InterruptedException, IOException;
default void expireSession(ZooKeeper zk) throws Exception {
diff --git a/bookkeeper-slogger/api/build.gradle b/bookkeeper-slogger/api/build.gradle
new file mode 100644
index 00000000000..2c232565fad
--- /dev/null
+++ b/bookkeeper-slogger/api/build.gradle
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+plugins {
+ id 'java'
+}
+
+dependencies {
+ testImplementation depLibs.hamcrest
+ testImplementation depLibs.junit
+}
+
+jar.archiveBaseName = 'bookkeeper-slogger-api'
diff --git a/bookkeeper-slogger/api/pom.xml b/bookkeeper-slogger/api/pom.xml
new file mode 100644
index 00000000000..25ac98d1af3
--- /dev/null
+++ b/bookkeeper-slogger/api/pom.xml
@@ -0,0 +1,46 @@
+
+
+
+
+ 4.0.0
+
+ bookkeeper-slogger-parent
+ org.apache.bookkeeper
+ 4.15.0-SNAPSHOT
+ ..
+
+ org.apache.bookkeeper
+ bookkeeper-slogger-api
+ Apache BookKeeper :: Structured Logger :: API
+
+
+ io.netty
+ netty-bom
+ 4.1.74.Final
+ pom
+ import
+
+
+ com.google.guava
+ guava-bom
+ 31.0.1-jre
+ pom
+ import
+
+
+
diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/AbstractSlogger.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/AbstractSlogger.java
new file mode 100644
index 00000000000..55ca52d38bc
--- /dev/null
+++ b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/AbstractSlogger.java
@@ -0,0 +1,275 @@
+/*
+ * 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.slogger;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.BiConsumer;
+
+/**
+ * Abstract implementation of slogger. Keeps track of key value pairs.
+ */
+public abstract class AbstractSlogger implements Slogger, Iterable