diff --git a/bookkeeper-server/pom.xml b/bookkeeper-server/pom.xml index da482b9c5ab..3fd415c4bc2 100644 --- a/bookkeeper-server/pom.xml +++ b/bookkeeper-server/pom.xml @@ -40,16 +40,6 @@ bookkeeper-proto ${project.parent.version} - - org.apache.bookkeeper - bookkeeper-slogger-slf4j - ${project.parent.version} - - - org.apache.bookkeeper - bookkeeper-slogger-api - ${project.parent.version} - org.apache.bookkeeper bookkeeper-tools-framework diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java index 58ac98fec53..5e122ae0091 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectCompactionEntryLog.java @@ -24,6 +24,7 @@ import static org.apache.bookkeeper.bookie.TransactionalEntryLogCompactor.COMPACTING_SUFFIX; import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import java.io.File; @@ -34,7 +35,6 @@ 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; /** @@ -43,7 +43,7 @@ public abstract class DirectCompactionEntryLog implements CompactionEntryLog { protected final int srcLogId; protected final int dstLogId; - protected final Slogger slog; + protected final Logger log; protected final File compactingFile; protected final File compactedFile; @@ -57,10 +57,10 @@ static CompactionEntryLog newLog(int srcLogId, BufferPool writeBuffers, NativeIO nativeIO, ByteBufAllocator allocator, - Slogger slog) throws IOException { + Logger log) throws IOException { return new WritingDirectCompactionEntryLog( srcLogId, dstLogId, ledgerDir, maxFileSize, - writeExecutor, writeBuffers, nativeIO, allocator, slog); + writeExecutor, writeBuffers, nativeIO, allocator, log); } static CompactionEntryLog recoverLog(int srcLogId, @@ -71,15 +71,15 @@ static CompactionEntryLog recoverLog(int srcLogId, NativeIO nativeIO, ByteBufAllocator allocator, OpStatsLogger readBlockStats, - Slogger slog) { + Logger log) { return new RecoveredDirectCompactionEntryLog(srcLogId, dstLogId, ledgerDir, readBufferSize, - maxSaneEntrySize, nativeIO, allocator, readBlockStats, slog); + maxSaneEntrySize, nativeIO, allocator, readBlockStats, log); } private DirectCompactionEntryLog(int srcLogId, int dstLogId, File ledgerDir, - Slogger slog) { + Logger log) { compactingFile = compactingFile(ledgerDir, dstLogId); compactedFile = compactedFile(ledgerDir, dstLogId, srcLogId); completeFile = DirectEntryLogger.logFile(ledgerDir, dstLogId); @@ -87,7 +87,11 @@ private DirectCompactionEntryLog(int srcLogId, this.srcLogId = srcLogId; this.dstLogId = dstLogId; - this.slog = slog.kv("dstLogId", dstLogId).kv("srcLogId", srcLogId).ctx(DirectCompactionEntryLog.class); + this.log = Logger.get(DirectCompactionEntryLog.class).with() + .ctx(log) + .attr("dstLogId", dstLogId) + .attr("srcLogId", srcLogId) + .build(); } @Override @@ -95,13 +99,15 @@ public void abort() { try { Files.deleteIfExists(compactingFile.toPath()); } catch (IOException ioe) { - slog.kv("compactingFile", compactingFile).warn(Events.COMPACTION_ABORT_EXCEPTION, ioe); + log.warn().exception(ioe).attr("compactingFile", compactingFile) + .log("Compaction aborted"); } try { Files.deleteIfExists(compactedFile.toPath()); } catch (IOException ioe) { - slog.kv("compactedFile", compactedFile).warn(Events.COMPACTION_ABORT_EXCEPTION, ioe); + log.warn().exception(ioe).attr("compactedFile", compactedFile) + .log("Compaction aborted"); } } @@ -109,8 +115,8 @@ public void abort() { @Override public void makeAvailable() throws IOException { idempotentLink(compactedFile, completeFile); - slog.kv("compactedFile", compactedFile).kv("completeFile", completeFile) - .info(Events.COMPACTION_MAKE_AVAILABLE); + log.info().attr("compactedFile", compactedFile).attr("completeFile", completeFile) + .log("Making compacted log available"); } private static void idempotentLink(File src, File dst) throws IOException { @@ -132,15 +138,17 @@ public void finalizeAndCleanup() { try { Files.deleteIfExists(compactingFile.toPath()); } catch (IOException ioe) { - slog.kv("compactingFile", compactingFile).warn(Events.COMPACTION_DELETE_FAILURE, ioe); + log.warn().exception(ioe).attr("compactingFile", compactingFile) + .log("Failed to delete compaction artifact"); } try { Files.deleteIfExists(compactedFile.toPath()); } catch (IOException ioe) { - slog.kv("compactedFile", compactedFile).warn(Events.COMPACTION_DELETE_FAILURE, ioe); + log.warn().exception(ioe).attr("compactedFile", compactedFile) + .log("Failed to delete compaction artifact"); } - slog.info(Events.COMPACTION_COMPLETE); + log.info("Compaction complete"); } @Override @@ -168,15 +176,15 @@ private static class RecoveredDirectCompactionEntryLog extends DirectCompactionE NativeIO nativeIO, ByteBufAllocator allocator, OpStatsLogger readBlockStats, - Slogger slog) { - super(srcLogId, dstLogId, ledgerDir, slog); + Logger log) { + super(srcLogId, dstLogId, ledgerDir, log); this.allocator = allocator; this.nativeIO = nativeIO; this.readBufferSize = readBufferSize; this.maxSaneEntrySize = maxSaneEntrySize; this.readBlockStats = readBlockStats; - this.slog.info(Events.COMPACTION_LOG_RECOVERED); + this.log.info("Recovered partially-flushed compaction log"); } private IllegalStateException illegalOpException() { @@ -223,16 +231,16 @@ private static class WritingDirectCompactionEntryLog extends DirectCompactionEnt BufferPool writeBuffers, NativeIO nativeIO, ByteBufAllocator allocator, - Slogger slog) throws IOException { - super(srcLogId, dstLogId, ledgerDir, slog); + Logger log) throws IOException { + super(srcLogId, dstLogId, ledgerDir, log); this.writer = new WriterWithMetadata( new DirectWriter(dstLogId, compactingFile.toString(), maxFileSize, - writeExecutor, writeBuffers, nativeIO, slog), + writeExecutor, writeBuffers, nativeIO, log), new EntryLogMetadata(dstLogId), allocator); - this.slog.info(Events.COMPACTION_LOG_CREATED); + this.log.info("Created compaction log"); } @Override @@ -251,13 +259,13 @@ public void markCompacted() throws IOException { idempotentLink(compactingFile, compactedFile); if (!compactingFile.delete()) { - slog.kv("compactingFile", compactingFile) - .kv("compactedFile", compactedFile) - .info(Events.COMPACTION_DELETE_FAILURE); + log.info().attr("compactingFile", compactingFile) + .attr("compactedFile", compactedFile) + .log("Failed to delete compaction artifact"); } else { - slog.kv("compactingFile", compactingFile) - .kv("compactedFile", compactedFile) - .info(Events.COMPACTION_MARK_COMPACTED); + log.info().attr("compactingFile", compactingFile) + .attr("compactedFile", compactedFile) + .log("Marked compaction log as compacted"); } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java index 035981514e9..3ba8f541d41 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectEntryLogger.java @@ -28,6 +28,7 @@ import com.google.common.cache.Cache; import com.google.common.cache.CacheBuilder; import com.google.common.cache.RemovalListener; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.util.ReferenceCountUtil; @@ -56,7 +57,6 @@ import org.apache.bookkeeper.bookie.storage.EntryLogScanner; import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.common.util.nativeio.NativeIO; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.StatsLogger; import org.apache.bookkeeper.util.LedgerDirUtil; @@ -64,7 +64,7 @@ * DirectEntryLogger. */ public class DirectEntryLogger implements EntryLogger { - private final Slogger slog; + private final Logger log; private final File ledgerDir; private final EntryLogIds ids; private final ExecutorService writeExecutor; @@ -99,7 +99,7 @@ public DirectEntryLogger(File ledgerDir, int readBufferSize, int numReadThreads, int maxFdCacheTimeSeconds, - Slogger slogParent, + Logger logParent, StatsLogger stats) throws IOException { this.ledgerDir = ledgerDir; this.flushExecutor = flushExecutor; @@ -112,7 +112,10 @@ public DirectEntryLogger(File ledgerDir, this.maxSaneEntrySize = maxSaneEntrySize; this.readBufferSize = Buffer.nextAlignment(readBufferSize); this.ids = ids; - this.slog = slogParent.kv("directory", ledgerDir).ctx(DirectEntryLogger.class); + this.log = Logger.get(DirectEntryLogger.class).with() + .ctx(logParent) + .attr("directory", ledgerDir) + .build(); this.stats = new DirectEntryLoggerStats(stats); @@ -129,28 +132,30 @@ public DirectEntryLogger(File ledgerDir, // 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); + log.error() + .attr("reason", + "perThreadBufferSize lower than readBufferSize (causes immediate reader cache eviction)") + .attr("totalReadBufferSize", totalReadBufferSize) + .attr("totalNumReadThreads", numReadThreads) + .attr("readBufferSize", readBufferSize) + .attr("perThreadBufferSize", perThreadBufferSize) + .log("Entry logger 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); + log.info() + .attr("maxFileSize", maxFileSize) + .attr("maxSaneEntrySize", maxSaneEntrySize) + .attr("totalWriteBufferSize", totalWriteBufferSize) + .attr("singleWriteBufferSize", singleWriteBufferSize) + .attr("totalReadBufferSize", totalReadBufferSize) + .attr("readBufferSize", readBufferSize) + .attr("perThreadBufferSize", perThreadBufferSize) + .attr("maxCachedReadersPerThread", maxCachedReadersPerThread) + .attr("maxCachedReaders", maxCachedReaders) + .log("Entry logger created"); this.caches = ThreadLocal.withInitial(() -> { RemovalListener rl = (notification) -> { @@ -158,7 +163,7 @@ public DirectEntryLogger(File ledgerDir, notification.getValue().close(); this.stats.getCloseReaderCounter().inc(); } catch (IOException ioe) { - slog.kv("logID", notification.getKey()).error(Events.READER_CLOSE_ERROR); + log.error().attr("logID", notification.getKey()).log("Failed to close entry log reader"); } }; Cache cache = CacheBuilder.newBuilder() @@ -190,7 +195,7 @@ public long addEntry(long ledgerId, ByteBuf buf) throws IOException { curWriter = new WriterWithMetadata(newDirectWriter(newId), new EntryLogMetadata(newId), allocator); - slog.kv("newLogId", newId).info(Events.LOG_ROLL); + log.info().attr("newLogId", newId).log("Rolling to new log file"); } offset = curWriter.addEntry(ledgerId, buf); @@ -378,7 +383,8 @@ 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); + log.info().attr("file", file).attr("logId", entryLogId).attr("result", result) + .log("Log file deleted"); return result; } @@ -402,8 +408,8 @@ public EntryLogMetadata getEntryLogMetadata(long entryLogId, AbstractLogCompacto try { return readEntryLogIndex(entryLogId); } catch (IOException e) { - slog.kv("entryLogId", entryLogId).kv("reason", e.getMessage()) - .info(Events.READ_METADATA_FALLBACK); + log.info().attr("entryLogId", entryLogId).attr("reason", e.getMessage()) + .log("Falling back to scanning log for metadata"); return scanEntryLogMetadata(entryLogId, throttler); } } @@ -450,7 +456,7 @@ LogReader newDirectReader(int logId) throws IOException { private LogWriter newDirectWriter(int newId) throws IOException { unflushedLogs.add(newId); LogWriter writer = new DirectWriter(newId, logFilename(ledgerDir, newId), maxFileSize, - writeExecutor, writeBuffers, nativeIO, slog); + writeExecutor, writeBuffers, nativeIO, log); ByteBuf buf = allocator.buffer(Buffer.ALIGNMENT); try { Header.writeEmptyHeader(buf); @@ -475,7 +481,7 @@ public CompactionEntryLog newCompactionLog(long srcLogId) throws IOException { int dstLogId = ids.nextId(); return DirectCompactionEntryLog.newLog((int) srcLogId, dstLogId, ledgerDir, maxFileSize, writeExecutor, writeBuffers, - nativeIO, allocator, slog); + nativeIO, allocator, log); } @Override @@ -490,7 +496,7 @@ public Collection incompleteCompactionLogs() { try { Files.deleteIfExists(f.toPath()); } catch (IOException ioe) { - slog.kv("file", f).warn(Events.COMPACTION_DELETE_FAILURE); + log.warn().attr("file", f).log("Failed to delete compaction artifact"); } } @@ -503,7 +509,7 @@ public Collection incompleteCompactionLogs() { readBufferSize, maxSaneEntrySize, nativeIO, allocator, stats.getReadBlockStats(), - slog)); + log)); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java index 20a3d855b4a..7a491df0e0f 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/DirectWriter.java @@ -24,6 +24,7 @@ import static com.google.common.base.Preconditions.checkState; import static org.apache.bookkeeper.common.util.ExceptionMessageHelper.exMsg; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import java.io.IOException; import java.util.ArrayList; @@ -33,7 +34,6 @@ 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 { @@ -45,7 +45,7 @@ class DirectWriter implements LogWriter { final ExecutorService writeExecutor; final Object bufferLock = new Object(); final List> outstandingWrites = new ArrayList>(); - final Slogger slog; + final Logger log; Buffer nativeBuffer; long offset; private static volatile boolean useFallocate = true; @@ -55,13 +55,13 @@ class DirectWriter implements LogWriter { long maxFileSize, ExecutorService writeExecutor, BufferPool bufferPool, - NativeIO nativeIO, Slogger slog) throws IOException { + NativeIO nativeIO, Logger log) throws IOException { checkArgument(maxFileSize > 0, "Max file size (%d) must be positive"); this.id = id; this.filename = filename; this.writeExecutor = writeExecutor; this.nativeIO = nativeIO; - this.slog = slog.ctx(DirectWriter.class); + this.log = Logger.get(DirectWriter.class).with().ctx(log).build(); offset = 0; @@ -78,7 +78,7 @@ class DirectWriter implements LogWriter { if (useFallocate) { if (!SystemUtils.IS_OS_LINUX) { disableUseFallocate(); - this.slog.warn(Events.FALLOCATE_NOT_AVAILABLE); + this.log.warn("Fallocate not available"); } else { try { int ret = nativeIO.fallocate(fd, NativeIO.FALLOC_FL_ZERO_RANGE, 0, maxFileSize); @@ -87,10 +87,11 @@ class DirectWriter implements LogWriter { // fallocate(2) is not supported on all filesystems. Since this is an optimization, disable // subsequent usage instead of failing the operation. disableUseFallocate(); - this.slog.kv("message", ex.getMessage()) - .kv("file", filename) - .kv("errno", ex.getErrno()) - .warn(Events.FALLOCATE_NOT_AVAILABLE); + this.log.warn() + .exceptionMessage(ex) + .attr("file", filename) + .attr("errno", ex.getErrno()) + .log("Fallocate not available"); } } } diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/EntryLogIdsImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/EntryLogIdsImpl.java index 2b63ec74f28..b5cabfd4834 100644 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/EntryLogIdsImpl.java +++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/EntryLogIdsImpl.java @@ -20,6 +20,7 @@ */ package org.apache.bookkeeper.bookie.storage.directentrylogger; +import io.github.merlimat.slog.Logger; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -27,7 +28,6 @@ import java.util.concurrent.TimeUnit; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.EntryLogIds; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.util.LedgerDirUtil; import org.apache.commons.lang3.tuple.Pair; /** @@ -37,14 +37,14 @@ public class EntryLogIdsImpl implements EntryLogIds { private final LedgerDirsManager ledgerDirsManager; - private final Slogger slog; + private final Logger log; private int nextId; private int maxId; public EntryLogIdsImpl(LedgerDirsManager ledgerDirsManager, - Slogger slog) throws IOException { + Logger log) throws IOException { this.ledgerDirsManager = ledgerDirsManager; - this.slog = slog.ctx(EntryLogIdsImpl.class); + this.log = Logger.get(EntryLogIdsImpl.class).with().ctx(log).build(); findLargestGap(); } @@ -74,10 +74,10 @@ private void findLargestGap() throws IOException { Pair gap = LedgerDirUtil.findLargestGap(currentIds); nextId = gap.getLeft(); maxId = gap.getRight(); - 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); + log.info().attr("dirs", ledgerDirsManager.getAllLedgerDirs()) + .attr("nextId", nextId) + .attr("maxId", maxId) + .attr("durationMs", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)) + .log("Entry log ID candidates selected"); } } 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 deleted file mode 100644 index 8f473c0f3ff..00000000000 --- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/directentrylogger/Events.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - * - */ -package org.apache.bookkeeper.bookie.storage.directentrylogger; - -/** - * Events. - */ -public enum Events { - /** - * Fallocate is not available on this host. This generally indicates that the process is running on a - * non-Linux operating system. The lack of fallocate means that the filesystem will have to do more - * bookkeeping as data is written to the file, which will slow down writes. - */ - FALLOCATE_NOT_AVAILABLE, - - /** - * EntryLog ID candidates selected. These are the set entry log ID that subsequent entry log files - * will use. To find the candidates, the bookie lists all the log ids which have already been used, - * and finds the longest contiguous block of free ids. Over the lifetime of a bookie, a log id can - * be reused. This is not a problem, as the ids are only referenced from the index, and an - * entry log file will not be deleted if there are still references to it in the index. - * Generally candidates are selected at bookie boot, but they may also be selected at a later time - * if the current set of candidates is depleted. - */ - ENTRYLOG_IDS_CANDIDATES_SELECTED, - - /** - * The entrylogger({@link org.apache.bookkeeper.bookie.storage.EntryLogger}) has been created. - * This occurs during bookie bootup, and the same entry logger will be used for the duration of - * the bookie process's lifetime. - */ - ENTRYLOGGER_CREATED, - - /** - * The entrylogger has been configured in a way that will likely result in errors during operation. - */ - ENTRYLOGGER_MISCONFIGURED, - - /** - * The entrylogger has started writing 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/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java index 69964c8f81f..6e9f317cbd5 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 @@ -27,6 +27,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.util.concurrent.DefaultThreadFactory; @@ -64,7 +65,6 @@ 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; @@ -213,7 +213,7 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le conf, DIRECT_IO_ENTRYLOGGER_MAX_FD_CACHE_TIME_SECONDS, DEFAULT_DIRECT_IO_MAX_FD_CACHE_TIME_SECONDS); - Slf4jSlogger slog = new Slf4jSlogger(DbLedgerStorage.class); + Logger log = Logger.get(DbLedgerStorage.class); entryLoggerWriteExecutor = Executors.newSingleThreadExecutor( new DefaultThreadFactory("EntryLoggerWrite")); entryLoggerFlushExecutor = Executors.newSingleThreadExecutor( @@ -224,7 +224,7 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le numReadThreads = conf.getServerNumIOThreads(); } - entrylogger = new DirectEntryLogger(ledgerDir, new EntryLogIdsImpl(ldm, slog), + entrylogger = new DirectEntryLogger(ledgerDir, new EntryLogIdsImpl(ldm, log), new NativeIOImpl(), allocator, entryLoggerWriteExecutor, entryLoggerFlushExecutor, conf.getEntryLogSizeLimit(), @@ -234,7 +234,7 @@ public void initialize(ServerConfiguration conf, LedgerManager ledgerManager, Le readBufferSize, numReadThreads, maxFdCacheTimeSeconds, - slog, statsLogger); + log, statsLogger); } else { entrylogger = new DefaultEntryLogger(conf, ldm, null, statsLogger, allocator); } 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 01d7e80f10c..ad2f0c95480 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 @@ -39,6 +39,8 @@ import static org.mockito.Mockito.when; import static org.mockito.MockitoAnnotations.openMocks; +import io.github.merlimat.slog.Event; +import io.github.merlimat.slog.Logger; import java.io.File; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -47,7 +49,6 @@ 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; @@ -64,7 +65,7 @@ */ @SuppressWarnings("deprecation") public class GarbageCollectorThreadTest { - private static final Slogger slog = Slogger.CONSOLE; + private static final Logger LOG = Logger.get(GarbageCollectorThreadTest.class); private final TmpDirs tmpDirs = new TmpDirs(); @@ -125,17 +126,17 @@ public void testCalculateUsageBucket() { double usage = ((double) item / (double) items); int index = mockGCThread.calculateUsageIndex(numBuckets, usage); assertFalse("Boundary condition exceeded", index < 0 || index >= numBuckets); - slog.kv("usage", usage) - .kv("index", index) - .info("Mapped usage to index"); + LOG.info().attr("usage", usage) + .attr("index", index) + .log("Mapped usage to index"); usageBuckets[index]++; } - Slogger sl = slog.ctx(); + Event ev = LOG.info(); for (int i = 0; i < numBuckets; i++) { - sl = sl.kv(bucketNames[i], usageBuckets[i]); + ev = ev.attr(bucketNames[i], usageBuckets[i]); } - sl.info("Compaction: entry log usage buckets"); + ev.log("Compaction: entry log usage buckets"); int sum = 0; for (int i = 0; i < numBuckets; i++) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java index f88e3883af5..d6ab899cbf7 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/EntryLogTestUtils.java @@ -24,6 +24,7 @@ import static org.hamcrest.Matchers.equalTo; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; @@ -35,7 +36,6 @@ import org.apache.bookkeeper.bookie.storage.directentrylogger.EntryLogIdsImpl; 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; @@ -43,7 +43,7 @@ * EntryLogTestUtils. */ public class EntryLogTestUtils { - private static final Slogger slog = Slogger.CONSOLE; + private static final Logger LOG = Logger.get(EntryLogTestUtils.class); public static LedgerDirsManager newDirsManager(File... ledgerDir) throws Exception { return new LedgerDirsManager( @@ -62,7 +62,7 @@ public static DirectEntryLogger newDirectEntryLogger(int logSizeLimit, File ledg curDir.mkdirs(); return new DirectEntryLogger( - curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), + curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG), new NativeIOImpl(), ByteBufAllocator.DEFAULT, MoreExecutors.newDirectExecutorService(), @@ -74,7 +74,7 @@ curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE); + LOG, NullStatsLogger.INSTANCE); } public static int logIdFromLocation(long location) { diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java index 56d3927dfe0..07674e0ffdc 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLogger.java @@ -30,6 +30,7 @@ import static org.hamcrest.Matchers.greaterThan; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.util.ReferenceCountUtil; @@ -46,7 +47,6 @@ import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; import org.junit.jupiter.api.AfterEach; @@ -61,7 +61,7 @@ @Slf4j @DisabledOnOs(OS.WINDOWS) public class TestDirectEntryLogger { - private final Slogger slog = Slogger.CONSOLE; + private final Logger log = Logger.get(TestDirectEntryLogger.class); private static final long ledgerId1 = 1234; @@ -95,7 +95,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { long loc1 = elog.addEntry(ledgerId1, e1.slice()); int logId1 = logIdFromLocation(loc1); assertThat(logId1, equalTo(1)); @@ -133,7 +133,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { long loc1 = elog.addEntry(ledgerId1, e1.slice()); long loc2 = elog.addEntry(ledgerId1, e2.slice()); elog.flush(); @@ -178,7 +178,7 @@ curDir, new MockEntryLogIds(), maxFileSize, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE) { + log, NullStatsLogger.INSTANCE) { @Override LogReader newDirectReader(int logId) throws IOException { outstandingReaders.incrementAndGet(); @@ -239,7 +239,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1); loc2 = elog.addEntry(ledgerId2, e2); loc3 = elog.addEntry(ledgerId1, e3); @@ -258,7 +258,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { int logId = logIdFromLocation(loc1); assertThat(logId, equalTo(logIdFromLocation(loc2))); assertThat(logId, equalTo(logIdFromLocation(loc3))); @@ -307,7 +307,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = writer.addEntry(ledgerId1, e1); loc2 = writer.addEntry(ledgerId2, e2); loc3 = writer.addEntry(ledgerId1, e3); @@ -326,7 +326,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { int logId = logIdFromLocation(loc1); try { reader.readEntryLogIndex(logId); @@ -366,7 +366,7 @@ curDir, new MockEntryLogIds(), 16 * 1024 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { for (int i = 0; i < ledgerCount; i++) { long loc = writer.addEntry(i, makeEntry(i, 1L, 1000)); if (lastLoc >= 0) { @@ -390,7 +390,7 @@ curDir, new MockEntryLogIds(), 16 * 1024 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { int logId = logIdFromLocation(lastLoc); EntryLogMetadata metaRead = reader.readEntryLogIndex(logId); @@ -436,7 +436,7 @@ curDir, new MockEntryLogIds(), 32 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE); + log, 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 @@ -503,7 +503,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { long loc1 = elog.addEntry(ledgerId1, e1.slice()); int logId1 = logIdFromLocation(loc1); assertThat(logId1, equalTo(1)); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java index 92d332c075f..42b312e92a0 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectEntryLoggerCompat.java @@ -29,6 +29,7 @@ import static org.hamcrest.Matchers.not; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; @@ -41,7 +42,6 @@ import org.apache.bookkeeper.bookie.storage.MockEntryLogIds; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; import org.apache.bookkeeper.util.DiskChecker; @@ -55,7 +55,7 @@ */ @DisabledOnOs(OS.WINDOWS) public class TestDirectEntryLoggerCompat { - private final Slogger slog = Slogger.CONSOLE; + private final Logger log = Logger.get(TestDirectEntryLoggerCompat.class); private static final long ledgerId1 = 1234; private static final long ledgerId2 = 4567; @@ -92,7 +92,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1.slice()); loc2 = elog.addEntry(ledgerId1, e2.slice()); loc3 = elog.addEntry(ledgerId1, e3.slice()); @@ -134,7 +134,7 @@ public void testDirectCanReadLegacy() throws Exception { 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { assertEntryEquals(elog.readEntry(ledgerId1, 1L, loc1), e1); assertEntryEquals(elog.readEntry(ledgerId1, 2L, loc2), e2); assertEntryEquals(elog.readEntry(ledgerId1, 3L, loc3), e3); @@ -165,7 +165,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1.slice()); loc2 = elog.addEntry(ledgerId1, e2.slice()); loc3 = elog.addEntry(ledgerId1, e3.slice()); @@ -204,7 +204,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1); loc2 = elog.addEntry(ledgerId2, e2); loc3 = elog.addEntry(ledgerId1, e3); @@ -255,7 +255,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1); loc2 = elog.addEntry(ledgerId2, e2); loc3 = elog.addEntry(ledgerId1, e3); @@ -312,7 +312,7 @@ curDir, new MockEntryLogIds(), 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { int logId = logIdFromLocation(loc1); assertThat(logId, equalTo(logIdFromLocation(loc2))); assertThat(logId, equalTo(logIdFromLocation(loc3))); @@ -371,7 +371,7 @@ public void testCompatFromDirectToDefaultToDirectLogger() throws Exception { 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc1 = elog.addEntry(ledgerId1, e1.slice()); loc2 = elog.addEntry(ledgerId1, e2.slice()); loc3 = elog.addEntry(ledgerId1, e3.slice()); @@ -436,7 +436,7 @@ public void testCompatFromDirectToDefaultToDirectLogger() throws Exception { 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc6 = elog.addEntry(ledgerId1, e6.slice()); loc7 = elog.addEntry(ledgerId1, e7.slice()); elog.flush(); @@ -534,7 +534,7 @@ public void testCompatFromDefaultToDirectToDefaultToDirectLogger() throws Except 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc3 = elog.addEntry(ledgerId1, e3.slice()); loc4 = elog.addEntry(ledgerId1, e4.slice()); loc5 = elog.addEntry(ledgerId1, e5.slice()); @@ -604,7 +604,7 @@ public void testCompatFromDefaultToDirectToDefaultToDirectLogger() throws Except 64 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE)) { + log, NullStatsLogger.INSTANCE)) { loc8 = elog.addEntry(ledgerId1, e8.slice()); loc9 = elog.addEntry(ledgerId1, e9.slice()); elog.flush(); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java index 03bd276e127..04cd4a0c429 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestDirectReader.java @@ -25,6 +25,7 @@ import static org.hamcrest.Matchers.equalTo; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; @@ -40,7 +41,6 @@ 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; @@ -270,7 +270,7 @@ public void testReadEntries() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), Logger.get(TestDirectReader.class))) { for (int i = 0; i < 1000; i++) { ByteBuf bb = Unpooled.buffer(entrySize); int pattern = 0xbeef + i; @@ -317,7 +317,7 @@ public int fallocate(int fd, int mode, long offset, long len) try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE); + buffers, new NativeIOImpl(), Logger.get(TestDirectReader.class)); LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), ByteBufAllocator.DEFAULT, new NativeIOImpl(), Buffer.ALIGNMENT, @@ -355,7 +355,7 @@ public void testReadFromFileBeingWrittenReadInPreallocated() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE); + buffers, new NativeIOImpl(), Logger.get(TestDirectReader.class)); LogReader reader = new DirectReader(1234, logFilename(ledgerDir, 1234), ByteBufAllocator.DEFAULT, new NativeIOImpl(), Buffer.ALIGNMENT, @@ -406,7 +406,7 @@ public int fallocate(int fd, int mode, long offset, long len) ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT * 10, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), Logger.get(TestDirectReader.class))) { ByteBuf b1 = Unpooled.buffer(entrySize); TestBuffer.fillByteBuf(b1, 0xfeedfeed); int offset1 = writer.writeDelimited(b1); @@ -455,7 +455,7 @@ public void testLargeEntry() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT * 8, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 20, MoreExecutors.newDirectExecutorService(), buffers, new NativeIOImpl(), - Slogger.CONSOLE)) { + Logger.get(TestDirectReader.class))) { ByteBuf b1 = Unpooled.buffer(entrySize); TestBuffer.fillByteBuf(b1, 0xfeedfeed); offset1 = writer.writeDelimited(b1); @@ -499,7 +499,7 @@ private static void writeFileWithPattern(File directory, int logId, try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(logId, logFilename(directory, logId), fileSize, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), Logger.get(TestDirectReader.class))) { for (int written = 0; written < fileSize; written += Buffer.ALIGNMENT) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); 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 index 4f1f3033ea9..a855f74037f 100644 --- 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 @@ -25,6 +25,7 @@ import static org.hamcrest.Matchers.equalTo; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import io.netty.buffer.Unpooled; @@ -38,7 +39,6 @@ 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.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; @@ -51,7 +51,7 @@ */ @DisabledOnOs(OS.WINDOWS) public class TestDirectWriter { - private static final Slogger slog = Slogger.CONSOLE; + private static final Logger LOG = Logger.get(TestDirectWriter.class); private final TmpDirs tmpDirs = new TmpDirs(); private final ExecutorService writeExecutor = Executors.newSingleThreadExecutor(); @@ -69,7 +69,7 @@ public void testWriteAtAlignment() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); TestBuffer.fillByteBuf(bb, 0xdededede); writer.writeAt(1234, bb); @@ -84,7 +84,7 @@ public void testWriteAlignmentSize() throws Exception { Assertions.assertThrows(IllegalArgumentException.class, () -> { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf bb = Unpooled.buffer(123); TestBuffer.fillByteBuf(bb, 0xdededede); writer.writeAt(0, bb); @@ -98,7 +98,7 @@ public void testWriteAlignedNotAtStart() throws Exception { File ledgerDir = tmpDirs.createNew("writeAlignment", "logs"); try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); TestBuffer.fillByteBuf(bb, 0xdededede); writer.writeAt(Buffer.ALIGNMENT * 2, bb); @@ -113,7 +113,7 @@ public void testFlushingWillWaitForBuffer() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, 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)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT / 2); TestBuffer.fillByteBuf(bb, 0xdededede); writer.writeDelimited(bb); @@ -140,7 +140,7 @@ public int pwrite(int fd, long pointer, int count, long offset) throws NativeIOE Assertions.assertThrows(IOException.class, () -> { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, io, Slogger.CONSOLE)) { + buffers, io, LOG)) { for (int i = 0; i < 10; i++) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT / 2); TestBuffer.fillByteBuf(bb, 0xdededede); @@ -170,7 +170,7 @@ public int pwrite(int fd, long pointer, int count, long offset) throws NativeIOE Assertions.assertThrows(IOException.class, () -> { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, 1 << 14, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, io, Slogger.CONSOLE)) { + buffers, io, LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); TestBuffer.fillByteBuf(bb, 0xdededede); writer.writeAt(0, bb); @@ -184,7 +184,7 @@ public void testWriteWithPadding() throws Exception { File ledgerDir = tmpDirs.createNew("paddingWrite", "logs"); try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, 1 << 14, 8); LogWriter writer = new DirectWriter(5678, logFilename(ledgerDir, 5678), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); TestBuffer.fillByteBuf(bb, 0xdededede); bb.writerIndex(123); @@ -213,7 +213,7 @@ public void testWriteBlocksFlush() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, 1 << 14, 8); LogWriter writer = new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { CompletableFuture blocker = new CompletableFuture<>(); writeExecutor.submit(() -> { blocker.join(); @@ -254,7 +254,7 @@ public void testFailsToOpen() throws Exception { Assertions.assertThrows(IOException.class, () -> { new DirectWriter(1234, logFilename(ledgerDir, 1234), 1 << 30, MoreExecutors.newDirectExecutorService(), - buffers, new NativeIOImpl(), Slogger.CONSOLE); + buffers, new NativeIOImpl(), LOG); }); } } @@ -272,7 +272,7 @@ public int fallocate(int fd, int mode, long offset, long len) try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, 1 << 14, 8); LogWriter writer = new DirectWriter(3456, logFilename(ledgerDir, 3456), 1 << 24, writeExecutor, - buffers, nativeIO, Slogger.CONSOLE)) { + buffers, nativeIO, LOG)) { ByteBuf bb = Unpooled.buffer(Buffer.ALIGNMENT); TestBuffer.fillByteBuf(bb, 0xdeadbeef); @@ -296,7 +296,7 @@ public void testWriteAtIntLimit() throws Exception { LogWriter writer = new DirectWriter(3456, logFilename(ledgerDir, 3456), (long) Integer.MAX_VALUE + (Buffer.ALIGNMENT * 100), writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), LOG)) { ByteBuf b1 = Unpooled.buffer(Buffer.ALIGNMENT - (Integer.BYTES * 2) - 1); TestBuffer.fillByteBuf(b1, 0xdeadbeef); @@ -324,7 +324,7 @@ buffers, new NativeIOImpl(), Slogger.CONSOLE)) { 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"); + LOG.info().attr("filename", file.toString()).log("Reading log file"); ByteBuf byteBuf = Unpooled.buffer((int) file.length()); try (FileInputStream is = new FileInputStream(file)) { int bytesRead = is.read(bytes); diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestEntryLogIds.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestEntryLogIds.java index de34f17499a..670b5249a1d 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestEntryLogIds.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestEntryLogIds.java @@ -30,12 +30,12 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import com.google.common.collect.Lists; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import java.io.File; import org.apache.bookkeeper.bookie.LedgerDirsManager; import org.apache.bookkeeper.bookie.storage.EntryLogIds; import org.apache.bookkeeper.bookie.storage.EntryLogger; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.test.TmpDirs; import org.apache.bookkeeper.util.LedgerDirUtil; import org.apache.commons.lang3.tuple.Pair; @@ -46,7 +46,7 @@ * TestEntryLogIds. */ public class TestEntryLogIds { - private static final Slogger slog = Slogger.CONSOLE; + private static final Logger LOG = Logger.get(TestEntryLogIds.class); private final TmpDirs tmpDirs = new TmpDirs(); @@ -72,7 +72,7 @@ public void testNoStomping() throws Exception { highestSoFar = logId2; } - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId3 = ids.nextId(); assertThat(logId3, greaterThan(highestSoFar)); touchLog(ledgerDir, logId3); @@ -101,7 +101,7 @@ public void testNoStompingDirectStartsFirst() throws Exception { File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); int highestSoFar = -1; - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId1 = ids.nextId(); assertThat(logId1, greaterThan(highestSoFar)); touchLog(ledgerDir, logId1); @@ -122,7 +122,7 @@ public void testNoStompingDirectStartsFirst() throws Exception { } // reinitialize to pick up legacy - ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId4 = ids.nextId(); assertThat(logId4, greaterThan(highestSoFar)); touchLog(ledgerDir, logId4); @@ -143,7 +143,7 @@ public void testIdGenerator() throws Exception { //case 1: use root ledgerDirsManager LedgerDirsManager ledgerDirsManager = newDirsManager(ledgerDir1, ledgerDir2); - EntryLogIds ids1 = new EntryLogIdsImpl(ledgerDirsManager, slog); + EntryLogIds ids1 = new EntryLogIdsImpl(ledgerDirsManager, LOG); for (int i = 0; i < 10; i++) { int logId = ids1.nextId(); File log1 = new File(ledgerDir1 + "/current", logId + ".log"); @@ -151,7 +151,7 @@ public void testIdGenerator() throws Exception { assertEquals(logId, i); } - EntryLogIds ids2 = new EntryLogIdsImpl(ledgerDirsManager, slog); + EntryLogIds ids2 = new EntryLogIdsImpl(ledgerDirsManager, LOG); for (int i = 0; i < 10; i++) { int logId = ids2.nextId(); assertEquals(logId, 10 + i); @@ -160,7 +160,7 @@ public void testIdGenerator() throws Exception { // case 2: new LedgerDirsManager for per directory LedgerDirsManager ledgerDirsManager3 = newDirsManager(ledgerDir3); LedgerDirsManager ledgerDirsManager4 = newDirsManager(ledgerDir4); - EntryLogIds ids3 = new EntryLogIdsImpl(ledgerDirsManager3, slog); + EntryLogIds ids3 = new EntryLogIdsImpl(ledgerDirsManager3, LOG); for (int i = 0; i < 10; i++) { int logId = ids3.nextId(); File log1 = new File(ledgerDir3 + "/current", logId + ".log"); @@ -168,7 +168,7 @@ public void testIdGenerator() throws Exception { assertEquals(logId, i); } - EntryLogIds ids4 = new EntryLogIdsImpl(ledgerDirsManager4, slog); + EntryLogIds ids4 = new EntryLogIdsImpl(ledgerDirsManager4, LOG); for (int i = 0; i < 10; i++) { int logId = ids4.nextId(); assertEquals(logId, i); @@ -203,7 +203,7 @@ public void testMultiDirectory() throws Exception { highestSoFar = logId3; } - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir1, ledgerDir2, ledgerDir3), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir1, ledgerDir2, ledgerDir3), LOG); int logId4 = ids.nextId(); assertThat(logId4, greaterThan(highestSoFar)); touchLog(ledgerDir2, logId4); @@ -221,10 +221,10 @@ public void testMultiDirectory() throws Exception { @Test public void testWrapAround() throws Exception { File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); - new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); touchLog(ledgerDir, Integer.MAX_VALUE - 1); - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId = ids.nextId(); assertThat(logId, equalTo(0)); } @@ -236,11 +236,11 @@ public void testCompactingLogsNotConsidered() throws Exception { // the directory concurrently, the transactional rename will prevent data // loss. File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); - new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); touchLog(ledgerDir, 123); touchCompacting(ledgerDir, 129); - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId = ids.nextId(); assertThat(logId, equalTo(124)); } @@ -248,11 +248,11 @@ public void testCompactingLogsNotConsidered() throws Exception { @Test public void testCompactedLogsConsidered() throws Exception { File ledgerDir = tmpDirs.createNew("entryLogIds", "ledgers"); - new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); touchLog(ledgerDir, 123); touchCompacted(ledgerDir, 129, 123); - EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), slog); + EntryLogIds ids = new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG); int logId = ids.nextId(); assertThat(logId, equalTo(130)); } diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestMetadata.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestMetadata.java index e44a28cf8da..b407593765a 100644 --- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestMetadata.java +++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/directentrylogger/TestMetadata.java @@ -24,6 +24,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import java.io.File; @@ -31,7 +32,6 @@ import java.util.concurrent.Executors; import org.apache.bookkeeper.bookie.EntryLogMetadata; 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; @@ -60,7 +60,7 @@ public void testReadMetaFromHeader() throws Exception { try (BufferPool buffers = new BufferPool(new NativeIOImpl(), ByteBufAllocator.DEFAULT, Buffer.ALIGNMENT, 8); LogWriter writer = new DirectWriter(logId, logFilename(ledgerDir, logId), 1 << 24, writeExecutor, - buffers, new NativeIOImpl(), Slogger.CONSOLE)) { + buffers, new NativeIOImpl(), Logger.get(TestMetadata.class))) { long offset = 4096L; writer.position(offset); EntryLogMetadata entryLogMetadata = new EntryLogMetadata(logId); 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 index 3f072846f61..6269b18c626 100644 --- 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 @@ -35,6 +35,7 @@ import static org.hamcrest.Matchers.not; import com.google.common.util.concurrent.MoreExecutors; +import io.github.merlimat.slog.Logger; import io.netty.buffer.ByteBuf; import io.netty.buffer.ByteBufAllocator; import java.io.File; @@ -53,7 +54,6 @@ import org.apache.bookkeeper.bookie.storage.EntryLogger; import org.apache.bookkeeper.common.util.nativeio.NativeIOImpl; import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.bookkeeper.slogger.Slogger; import org.apache.bookkeeper.stats.NullStatsLogger; import org.apache.bookkeeper.test.TmpDirs; import org.junit.jupiter.api.AfterEach; @@ -66,7 +66,7 @@ */ @DisabledOnOs(OS.WINDOWS) public class TestTransactionalEntryLogCompactor { - private static final Slogger slog = Slogger.CONSOLE; + private static final Logger LOG = Logger.get(TestTransactionalEntryLogCompactor.class); private final TmpDirs tmpDirs = new TmpDirs(); private static final long deadLedger = 1L; @@ -537,7 +537,7 @@ private static DirectEntryLogger newDirectEntryLoggerCompactionOverride( curDir.mkdirs(); return new DirectEntryLogger( - curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), + curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), LOG), new NativeIOImpl(), ByteBufAllocator.DEFAULT, MoreExecutors.newDirectExecutorService(), @@ -549,7 +549,7 @@ curDir, new EntryLogIdsImpl(newDirsManager(ledgerDir), slog), 4 * 1024, // read buffer size 1, // numReadThreads 300, // max fd cache time in seconds - slog, NullStatsLogger.INSTANCE) { + LOG, NullStatsLogger.INSTANCE) { @Override public CompactionEntryLog newCompactionLog(long logToCompact) throws IOException { return override.apply(super.newCompactionLog(logToCompact)); diff --git a/bookkeeper-slogger/api/pom.xml b/bookkeeper-slogger/api/pom.xml deleted file mode 100644 index 59dd4459d9f..00000000000 --- a/bookkeeper-slogger/api/pom.xml +++ /dev/null @@ -1,27 +0,0 @@ - - - - 4.0.0 - - bookkeeper-slogger-parent - org.apache.bookkeeper - 4.18.0-SNAPSHOT - .. - - org.apache.bookkeeper - bookkeeper-slogger-api - Apache BookKeeper :: Structured Logger :: API - 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 deleted file mode 100644 index 55ca52d38bc..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/AbstractSlogger.java +++ /dev/null @@ -1,275 +0,0 @@ -/* - * 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 { - /** - * Levels at which slogger can slog. - */ - public enum Level { - INFO, - WARN, - ERROR - } - - private static final int MAX_DEPTH = 3; - private List parentCtx; - - private ThreadLocal> kvs = new ThreadLocal>() { - @Override - protected List initialValue() { - return new ArrayList<>(); - } - }; - private ThreadLocal> flattenedTls = ThreadLocal.withInitial(ArrayList::new); - - protected AbstractSlogger(Iterable parentCtx) { - List flattened = new ArrayList<>(); - flattenKeyValues(parentCtx.iterator(), (k, v) -> { - flattened.add(k); - flattened.add(v); - }); - this.parentCtx = Collections.unmodifiableList(flattened); - } - - protected abstract Slogger newSlogger(Optional> clazz, Iterable parent); - protected abstract void doLog(Level level, Enum event, String message, - Throwable throwable, List keyValues); - - private void flattenAndLog(Level level, Enum event, String message, - Throwable throwable) { - List flattened = flattenedTls.get(); - flattened.clear(); - - flattenKeyValues(this::addToFlattened); - doLog(level, event, message, throwable, flattened); - } - - @Override - public void info(String message) { - flattenAndLog(Level.INFO, null, message, null); - } - - @Override - public void info(String message, Throwable cause) { - flattenAndLog(Level.INFO, null, message, cause); - } - - @Override - public void info(Enum event) { - flattenAndLog(Level.INFO, event, null, null); - } - - @Override - public void info(Enum event, Throwable cause) { - flattenAndLog(Level.INFO, event, null, cause); - } - - @Override - public void warn(String message) { - flattenAndLog(Level.WARN, null, message, null); - } - - @Override - public void warn(String message, Throwable cause) { - flattenAndLog(Level.WARN, null, message, cause); - } - - @Override - public void warn(Enum event) { - flattenAndLog(Level.WARN, event, null, null); - } - - @Override - public void warn(Enum event, Throwable cause) { - flattenAndLog(Level.WARN, event, null, cause); - } - - @Override - public void error(String message) { - flattenAndLog(Level.ERROR, null, message, null); - } - - @Override - public void error(String message, Throwable cause) { - flattenAndLog(Level.ERROR, null, message, cause); - } - - @Override - public void error(Enum event) { - flattenAndLog(Level.ERROR, event, null, null); - } - - @Override - public void error(Enum event, Throwable cause) { - flattenAndLog(Level.ERROR, event, null, cause); - } - - @Override - public Slogger ctx() { - try { - return newSlogger(Optional.empty(), this); - } finally { - kvs.get().clear(); - } - } - - @Override - public Slogger ctx(Class clazz) { - try { - return newSlogger(Optional.of(clazz), this); - } finally { - kvs.get().clear(); - } - } - - @Override - public Iterator iterator() { - CtxIterator iterator = this.iterator.get(); - iterator.reset(); - return iterator; - } - - protected void clearCurrentCtx() { - kvs.get().clear(); - } - - private void addToFlattened(String key, String value) { - flattenedTls.get().add(key); - flattenedTls.get().add(value); - } - - protected void flattenKeyValues(BiConsumer consumer) { - Iterator iter = iterator(); - try { - flattenKeyValues(iter, consumer); - } finally { - kvs.get().clear(); - } - } - - public static void flattenKeyValues(Iterator iter, - BiConsumer consumer) { - while (iter.hasNext()) { - String key = iter.next().toString(); - if (!iter.hasNext()) { - return; // key without value - } - Object value = iter.next(); - - if (value instanceof Sloggable) { - addWithPrefix(key, (Sloggable) value, consumer, 0); - } else if (value.getClass().isArray()) { - consumer.accept(key, arrayToString(value)); - } else { - consumer.accept(key, value.toString()); - } - } - } - - @Override - public Slogger kv(Object key, Object value) { - kvs.get().add(key); - kvs.get().add(value); - return this; - } - - private static void addWithPrefix(String prefix, Sloggable value, - BiConsumer consumer, int depth) { - value.log(new SloggableAccumulator() { - @Override - public SloggableAccumulator kv(Object key, Object value) { - if (value instanceof Sloggable && depth < MAX_DEPTH) { - addWithPrefix(prefix + "." + key.toString(), - (Sloggable) value, consumer, depth + 1); - } else if (value.getClass().isArray()) { - consumer.accept(prefix + "." + key.toString(), arrayToString(value)); - } else { - consumer.accept(prefix + "." + key.toString(), value.toString()); - } - return this; - } - }); - } - - private static String arrayToString(Object o) { - if (o instanceof long[]) { - return Arrays.toString((long[]) o); - } else if (o instanceof int[]) { - return Arrays.toString((int[]) o); - } else if (o instanceof short[]) { - return Arrays.toString((short[]) o); - } else if (o instanceof char[]) { - return Arrays.toString((char[]) o); - } else if (o instanceof byte[]) { - return Arrays.toString((byte[]) o); - } else if (o instanceof boolean[]) { - return Arrays.toString((boolean[]) o); - } else if (o instanceof float[]) { - return Arrays.toString((float[]) o); - } else if (o instanceof double[]) { - return Arrays.toString((double[]) o); - } else if (o instanceof Object[]) { - return Arrays.toString((Object[]) o); - } else { - return o.toString(); - } - } - - private final ThreadLocal iterator = new ThreadLocal() { - @Override - protected CtxIterator initialValue() { - return new CtxIterator(); - } - }; - class CtxIterator implements Iterator { - int index = 0; - - private void reset() { - index = 0; - } - - @Override - public boolean hasNext() { - return index < (parentCtx.size() + kvs.get().size()); - } - - @Override - public Object next() { - int i = index++; - if (i < parentCtx.size()) { - return parentCtx.get(i); - } else { - i -= parentCtx.size(); - return kvs.get().get(i); - } - } - } -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/ConsoleSlogger.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/ConsoleSlogger.java deleted file mode 100644 index 74e53a4b8eb..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/ConsoleSlogger.java +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.time.ZoneOffset; -import java.time.ZonedDateTime; -import java.time.format.DateTimeFormatter; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -/** - * Simple slogger implementation which writes json to console. - */ -public class ConsoleSlogger extends AbstractSlogger { - private static final int MAX_STACKTRACE_ELEMENTS = 20; - private static final int MAX_CAUSES = 10; - private final Class clazz; - - ConsoleSlogger() { - this(ConsoleSlogger.class); - } - - ConsoleSlogger(Class clazz) { - this(clazz, Collections.emptyList()); - } - - ConsoleSlogger(Class clazz, Iterable parent) { - super(parent); - this.clazz = clazz; - } - - @Override - protected Slogger newSlogger(Optional> clazz, Iterable parent) { - return new ConsoleSlogger(clazz.orElse(ConsoleSlogger.class), parent); - } - - @Override - protected void doLog(Level level, Enum event, String message, - Throwable throwable, List keyValues) { - String nowAsISO = ZonedDateTime.now(ZoneOffset.UTC).format(DateTimeFormatter.ISO_INSTANT); - - StringBuilder builder = new StringBuilder(); - builder.append("{"); - keyValue(builder, "date", nowAsISO); - builder.append(","); - keyValue(builder, "level", level.toString()); - if (event != null) { - builder.append(","); - keyValue(builder, "event", event.toString()); - } - if (message != null) { - builder.append(","); - keyValue(builder, "message", message); - } - - for (int i = 0; i < keyValues.size(); i += 2) { - builder.append(","); - keyValue(builder, keyValues.get(i).toString(), keyValues.get(i + 1).toString()); - } - if (throwable != null) { - builder.append(","); - Throwable cause = throwable; - StringBuilder stacktrace = new StringBuilder(); - int causes = 0; - while (cause != null) { - stacktrace.append("[").append(cause.getMessage()).append("] at "); - int i = 0; - for (StackTraceElement element : cause.getStackTrace()) { - if (i++ > MAX_STACKTRACE_ELEMENTS) { - stacktrace.append("<|[frames omitted]"); - } - stacktrace.append("<|").append(element.toString()); - } - cause = cause.getCause(); - if (cause != null) { - if (causes++ > MAX_CAUSES) { - stacktrace.append(" [max causes exceeded] "); - break; - } else { - stacktrace.append(" caused by "); - } - } - } - keyValue(builder, "exception", stacktrace.toString()); - } - builder.append("}"); - - System.out.println(builder); - } - - private static void keyValue(StringBuilder sb, String key, String value) { - quotedAppend(sb, key); - sb.append(":"); - quotedAppend(sb, value); - } - - private static void quotedAppend(StringBuilder sb, String str) { - sb.append('"'); - for (int i = 0; i < str.length(); i++) { - char c = str.charAt(i); - if (c == '\\') { - sb.append("\\\\"); - } else if (c == '"') { - sb.append("\\\""); - } else if (c < ' ') { - sb.append(String.format("\\u%04X", (int) c)); - } else { - sb.append(c); - } - } - sb.append('"'); - } -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/NullSlogger.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/NullSlogger.java deleted file mode 100644 index 2c5302f80ab..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/NullSlogger.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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; - -class NullSlogger implements Slogger { - @Override - public Slogger kv(Object key, Object value) { - return this; - } - - @Override - public Slogger ctx() { - return this; - } - - @Override - public Slogger ctx(Class clazz) { - return this; - } - - @Override - public void info(String message) {} - @Override - public void info(String message, Throwable cause) {} - @Override - public void info(Enum event) {} - @Override - public void info(Enum event, Throwable cause) {} - - @Override - public void warn(String message) {} - @Override - public void warn(String message, Throwable cause) {} - @Override - public void warn(Enum event) {} - @Override - public void warn(Enum event, Throwable cause) {} - - @Override - public void error(String message) {} - @Override - public void error(String message, Throwable cause) {} - @Override - public void error(Enum event) {} - @Override - public void error(Enum event, Throwable cause) {} -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Sloggable.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Sloggable.java deleted file mode 100644 index 165dff5d682..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Sloggable.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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; - -/** - * Interface to be implemented by classes that want more control - * over how they are added to a structured log. - */ -public interface Sloggable { - SloggableAccumulator log(SloggableAccumulator accumulator); -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/SloggableAccumulator.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/SloggableAccumulator.java deleted file mode 100644 index 10f484b0b07..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/SloggableAccumulator.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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; - -/** - * Interface passed to Sloggable instances, with which they - * can add their own key/value pairs to the logged event. - */ -public interface SloggableAccumulator { - SloggableAccumulator kv(Object key, Object value); -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Slogger.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Slogger.java deleted file mode 100644 index f91b2b8f449..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/Slogger.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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; - -/** - * Event logging interface will support for key value pairs and reusable context. - */ -public interface Slogger { - Slogger kv(Object key, Object value); - - Slogger ctx(); - Slogger ctx(Class clazz); // <- should this be class or Logger? Logger requires some generics - - void info(String message); - void info(String message, Throwable cause); - void info(Enum event); - void info(Enum event, Throwable cause); - - void warn(String message); - void warn(String message, Throwable cause); - void warn(Enum event); - void warn(Enum event, Throwable cause); - - void error(String message); - void error(String message, Throwable cause); - void error(Enum event); - void error(Enum event, Throwable cause); - - Slogger NULL = new NullSlogger(); - Slogger CONSOLE = new ConsoleSlogger(); -} diff --git a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/package-info.java b/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/package-info.java deleted file mode 100644 index f132eafa8d2..00000000000 --- a/bookkeeper-slogger/api/src/main/java/org/apache/bookkeeper/slogger/package-info.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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. - */ - -/** - * Structured logging. - */ -package org.apache.bookkeeper.slogger; diff --git a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConcurrencyTest.java b/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConcurrencyTest.java deleted file mode 100644 index 07c28c37bfa..00000000000 --- a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConcurrencyTest.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * 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.Collections; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import org.junit.Test; - -/** - * Test concurrent access to slogger. - */ -public class ConcurrencyTest { - enum Events { - FOOBAR - } - - @Test - public void testConcurrentFlattening() throws Exception { - final int numThreads = 100; - final int numIterations = 10000; - - Slogger slog = new AbstractSlogger(Collections.emptyList()) { - @Override - public Slogger newSlogger(Optional> clazz, Iterable parent) { - return this; - } - @Override - public void doLog(Level level, Enum event, String message, - Throwable throwable, List keyValues) { - for (int i = 0; i < keyValues.size(); i += 2) { - if (!keyValues.get(i).equals(keyValues.get(i + 1))) { - - throw new RuntimeException("Concurrency error"); - } - } - } - }; - - ExecutorService executor = Executors.newFixedThreadPool(numThreads); - List> futures = new ArrayList<>(); - for (int i = 0; i < numThreads; i++) { - futures.add(executor.submit(() -> { - for (int j = 0; j < numIterations; j++) { - String value = "kv" + Thread.currentThread().getId() + "-" + j; - - slog.kv(value, value).info(Events.FOOBAR); - } - })); - } - - for (Future f : futures) { - f.get(60, TimeUnit.SECONDS); - } - } -} diff --git a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConsoleSloggerTest.java b/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConsoleSloggerTest.java deleted file mode 100644 index 0ca3612346c..00000000000 --- a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/ConsoleSloggerTest.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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 org.junit.Test; - -/** - * Test console slogger. - * Doesn't actually assert anything, but can be used to eyeball - */ -public class ConsoleSloggerTest { - enum Events { - FOOBAR, - BARFOO - }; - - @Test - public void testBasic() throws Exception { - ConsoleSlogger root = new ConsoleSlogger(); - root.kv("fo\"o", "ba\r \\").info(Events.FOOBAR); - } -} - diff --git a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/MockSlogger.java b/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/MockSlogger.java deleted file mode 100644 index f71465f88e1..00000000000 --- a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/MockSlogger.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * 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.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -/** - * Mock Slogger. - */ -public class MockSlogger extends AbstractSlogger { - List events = new ArrayList<>(); - - public MockSlogger() { - super(new ArrayList<>()); - } - - private MockSlogger(Iterable parentCtx) { - super(parentCtx); - } - - @Override - protected Slogger newSlogger(Optional> clazz, Iterable parentCtx) { - return new MockSlogger(parentCtx); - } - - @Override - protected void doLog(Level level, Enum event, String message, Throwable throwable, - List keyValues) { - Map tmpKvs = new HashMap<>(); - for (int i = 0; i < keyValues.size(); i += 2) { - tmpKvs.put(keyValues.get(i).toString(), keyValues.get(i + 1)); - } - events.add(new MockEvent(level, event, message, tmpKvs, throwable)); - } - - static class MockEvent { - private final Level level; - private final Enum event; - private final String message; - private final Map kvs; - private final Throwable throwable; - - MockEvent(Level level, Enum event, String message, - Map kvs, Throwable throwable) { - this.level = level; - this.event = event; - this.message = message; - this.kvs = kvs; - this.throwable = throwable; - } - - Level getLevel() { - return level; - } - Enum getEvent() { - return event; - } - String getMessage() { - return message; - } - Map getKeyValues() { - return kvs; - } - Throwable getThrowable() { - return throwable; - } - } -} diff --git a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/SloggerTest.java b/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/SloggerTest.java deleted file mode 100644 index 32bf663d7e2..00000000000 --- a/bookkeeper-slogger/api/src/test/java/org/apache/bookkeeper/slogger/SloggerTest.java +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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 static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.hasEntry; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.is; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import org.junit.Test; - -/** - * Test Slogger. - */ -public class SloggerTest { - enum Events { - FOOBAR, - BARFOO - }; - - @Test - public void testBasic() throws Exception { - MockSlogger root = new MockSlogger(); - root.kv("foo", 2324).kv("bar", 2342).info(Events.FOOBAR); - assertThat(root.events, hasSize(1)); - assertThat(root.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(root.events.get(0).getEvent(), is(Events.FOOBAR)); - assertThat(root.events.get(0).getKeyValues(), - allOf(hasEntry("foo", "2324"), - hasEntry("bar", "2342"))); - } - - @Test - public void testSloggable() throws Exception { - MockSlogger root = new MockSlogger(); - root.kv("fancy", new FancyClass(0, 2)).info(Events.FOOBAR); - assertThat(root.events, hasSize(1)); - assertThat(root.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(root.events.get(0).getEvent(), is(Events.FOOBAR)); - assertThat(root.events.get(0).getKeyValues(), - allOf(hasEntry("fancy.foo", "0"), - hasEntry("fancy.bar", "2"), - hasEntry("fancy.baz.baz", "123"))); - } - - @Test - public void testList() throws Exception { - MockSlogger root = new MockSlogger(); - List list = new ArrayList<>(); - list.add(1); - list.add(2); - root.kv("list", list).info(Events.FOOBAR); - - assertThat(root.events, hasSize(1)); - assertThat(root.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(root.events.get(0).getEvent(), is(Events.FOOBAR)); - assertThat(root.events.get(0).getKeyValues(), hasEntry("list", "[1, 2]")); - } - - @Test - public void testMap() throws Exception { - MockSlogger root = new MockSlogger(); - HashMap map = new LinkedHashMap<>(); - map.put(1, 3); - map.put(2, 4); - root.kv("map", map).info(Events.FOOBAR); - - assertThat(root.events, hasSize(1)); - assertThat(root.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(root.events.get(0).getEvent(), is(Events.FOOBAR)); - assertThat(root.events.get(0).getKeyValues(), hasEntry("map", "{1=3, 2=4}")); - } - - @Test - public void testArray() throws Exception { - MockSlogger root = new MockSlogger(); - String[] array = {"foo", "bar"}; - root.kv("array", array).info(Events.FOOBAR); - - assertThat(root.events, hasSize(1)); - assertThat(root.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(root.events.get(0).getEvent(), is(Events.FOOBAR)); - assertThat(root.events.get(0).getKeyValues(), hasEntry("array", "[foo, bar]")); - } - - @Test - public void testNestingLimit() throws Exception { - } - - @Test - public void testCtx() throws Exception { - MockSlogger root = new MockSlogger(); - MockSlogger withCtx = (MockSlogger) root.kv("ctx1", 1234).kv("ctx2", 4321).ctx(); - - withCtx.kv("someMore", 2345).info(Events.FOOBAR); - - assertThat(withCtx.events, hasSize(1)); - assertThat(withCtx.events.get(0).getLevel(), is(MockSlogger.Level.INFO)); - assertThat(withCtx.events.get(0).getEvent(), is(Events.FOOBAR)); - System.out.println("kvs " + withCtx.events.get(0).getKeyValues()); - assertThat(withCtx.events.get(0).getKeyValues(), - allOf(hasEntry("ctx1", "1234"), - hasEntry("ctx2", "4321"), - hasEntry("someMore", "2345"))); - } - - @Test - public void textCtxImmutableAfterCreation() throws Exception { - } - - static class FancyClass implements Sloggable { - int foo; - int bar; - OtherFancyClass baz; - - FancyClass(int foo, int bar) { - this.foo = foo; - this.bar = bar; - this.baz = new OtherFancyClass(123); - } - - @Override - public SloggableAccumulator log(SloggableAccumulator slogger) { - return slogger.kv("foo", foo) - .kv("bar", bar) - .kv("baz", baz); - } - } - - static class OtherFancyClass implements Sloggable { - int baz; - - OtherFancyClass(int baz) { - this.baz = baz; - } - - @Override - public SloggableAccumulator log(SloggableAccumulator slogger) { - return slogger.kv("baz", baz); - } - } -} diff --git a/bookkeeper-slogger/pom.xml b/bookkeeper-slogger/pom.xml deleted file mode 100644 index 6bd3d91f59f..00000000000 --- a/bookkeeper-slogger/pom.xml +++ /dev/null @@ -1,49 +0,0 @@ - - - - 4.0.0 - - org.apache.bookkeeper - bookkeeper - 4.18.0-SNAPSHOT - .. - - pom - bookkeeper-slogger-parent - Apache BookKeeper :: Structured Logger :: Parent - - - api - slf4j - - - - - - com.github.spotbugs - spotbugs-maven-plugin - - true - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - diff --git a/bookkeeper-slogger/slf4j/pom.xml b/bookkeeper-slogger/slf4j/pom.xml deleted file mode 100644 index e45b5848bfe..00000000000 --- a/bookkeeper-slogger/slf4j/pom.xml +++ /dev/null @@ -1,34 +0,0 @@ - - - - 4.0.0 - - bookkeeper-slogger-parent - org.apache.bookkeeper - 4.18.0-SNAPSHOT - .. - - org.apache.bookkeeper - bookkeeper-slogger-slf4j - Apache BookKeeper :: Structured Logger :: SLF4J Implementation - - - org.apache.bookkeeper - bookkeeper-slogger-api - ${project.parent.version} - - - diff --git a/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/Slf4jSlogger.java b/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/Slf4jSlogger.java deleted file mode 100644 index 81400227ab3..00000000000 --- a/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/Slf4jSlogger.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * 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.slf4j; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import org.apache.bookkeeper.slogger.AbstractSlogger; -import org.apache.bookkeeper.slogger.Slogger; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.slf4j.MDC; - -/** - * Slf4j implementation of slogger. - */ -public class Slf4jSlogger extends AbstractSlogger { - private ThreadLocal> mdcKeysTls = new ThreadLocal>() { - @Override - protected List initialValue() { - return new ArrayList<>(); - } - }; - - private final Logger log; - - public Slf4jSlogger(Class clazz) { - this(clazz, Collections.emptyList()); - } - - Slf4jSlogger() { - this(Slf4jSlogger.class); - } - - Slf4jSlogger(Class clazz, Iterable parent) { - super(parent); - this.log = LoggerFactory.getLogger(clazz); - } - - @Override - protected Slogger newSlogger(Optional> clazz, Iterable parent) { - return new Slf4jSlogger(clazz.orElse(Slf4jSlogger.class), parent); - } - - @Override - protected void doLog(Level level, Enum event, String message, - Throwable throwable, List keyValues) { - List mdcKeys = mdcKeysTls.get(); - mdcKeys.clear(); - try { - if (event != null) { - MDC.put("event", event.toString()); - mdcKeys.add("event"); - } - - for (int i = 0; i < keyValues.size(); i += 2) { - MDC.put(keyValues.get(i).toString(), keyValues.get(i + 1).toString()); - mdcKeys.add(keyValues.get(i).toString()); - } - - String msg = message == null ? event.toString() : message; - switch (level) { - case INFO: - log.info(msg); - break; - case WARN: - if (throwable != null) { - log.warn(msg, throwable); - } else { - log.warn(msg); - } - break; - default: - case ERROR: - if (throwable != null) { - log.error(msg, throwable); - } else { - log.error(msg); - } - break; - } - } finally { - for (String key : mdcKeys) { - MDC.remove(key); - } - mdcKeys.clear(); - } - } -} diff --git a/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/package-info.java b/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/package-info.java deleted file mode 100644 index 69ff1ed3eb9..00000000000 --- a/bookkeeper-slogger/slf4j/src/main/java/org/apache/bookkeeper/slogger/slf4j/package-info.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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. - */ - -/** - * Structured logging (slf4j implementation). - */ -package org.apache.bookkeeper.slogger.slf4j; diff --git a/bookkeeper-slogger/slf4j/src/test/java/org/apache/bookkeeper/slogger/slf4j/Slf4jTest.java b/bookkeeper-slogger/slf4j/src/test/java/org/apache/bookkeeper/slogger/slf4j/Slf4jTest.java deleted file mode 100644 index 848a05cfacf..00000000000 --- a/bookkeeper-slogger/slf4j/src/test/java/org/apache/bookkeeper/slogger/slf4j/Slf4jTest.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.slf4j; - -import org.apache.bookkeeper.slogger.Slogger; -import org.junit.Test; - -/** - * Test to eyeball slf4j output. - * Contains no asserts. - */ -public class Slf4jTest { - enum Events { - FOOBAR - } - @Test - public void testBasic() throws Exception { - Slogger slogger = new Slf4jSlogger(Slf4jTest.class); - slogger.kv("foo", 123).kv("bar", 432).info(Events.FOOBAR); - } -} diff --git a/pom.xml b/pom.xml index d18f145ed76..92eb173bac6 100644 --- a/pom.xml +++ b/pom.xml @@ -68,7 +68,6 @@ bookkeeper-dist shaded microbenchmarks - bookkeeper-slogger tests native-io testtools