> sets;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
index a353b7cf7ee..bdc1bf94649 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
@@ -26,6 +26,7 @@
import java.util.Map.Entry;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.Bookie;
import org.apache.bookkeeper.bookie.EntryLocation;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorage.Batch;
@@ -34,8 +35,6 @@
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.collections.ConcurrentLongHashSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Maintains an index of the entry locations in the EntryLogger.
@@ -43,6 +42,7 @@
* For each ledger multiple entries are stored in the same "record", represented
* by the {@link LedgerIndexPage} class.
*/
+@CustomLog
public class EntryLocationIndex implements Closeable {
private final KeyValueStorage locationsDb;
@@ -78,9 +78,10 @@ public long getLocation(long ledgerId, long entryId) throws IOException {
boolean operationSuccess = false;
try {
if (locationsDb.get(key.array, value.array) < 0) {
- if (log.isDebugEnabled()) {
- log.debug("Entry not found {}@{} in db index", ledgerId, entryId);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .log("Entry not found in db index");
return 0;
}
operationSuccess = true;
@@ -101,9 +102,7 @@ public long getLocation(long ledgerId, long entryId) throws IOException {
public long getLastEntryInLedger(long ledgerId) throws IOException {
if (deletedLedgers.contains(ledgerId)) {
// Ledger already deleted
- if (log.isDebugEnabled()) {
- log.debug("Ledger {} already deleted in db", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Ledger already deleted in db");
/**
* when Ledger already deleted,
* throw Bookie.NoEntryException same like the method
@@ -136,9 +135,10 @@ private long getLastEntryInLedgerInternal(long ledgerId) throws IOException {
long lastEntryId = ArrayUtil.getLong(entry.getKey(), 8);
if (foundLedgerId == ledgerId) {
- if (log.isDebugEnabled()) {
- log.debug("Found last page in storage db for ledger {} - last entry: {}", ledgerId, lastEntryId);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("lastEntryId", lastEntryId)
+ .log("Found last page in storage db for ledger");
return lastEntryId;
} else {
throw new Bookie.NoEntryException(ledgerId, -1);
@@ -161,9 +161,11 @@ public void addLocation(Batch batch, long ledgerId, long entryId, long location)
LongPairWrapper key = LongPairWrapper.get(ledgerId, entryId);
LongWrapper value = LongWrapper.get(location);
- if (log.isDebugEnabled()) {
- log.debug("Add location - ledger: {} -- entry: {} -- location: {}", ledgerId, entryId, location);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("location", location)
+ .log("Add location");
try {
batch.put(key.array, value.array);
@@ -174,16 +176,15 @@ public void addLocation(Batch batch, long ledgerId, long entryId, long location)
}
public void updateLocations(Iterable newLocations) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("Update locations -- {}", Iterables.size(newLocations));
- }
+ log.debug(e -> e.attr("count", Iterables.size(newLocations)).log("Update locations"));
try (Batch batch = newBatch()) {
// Update all the ledger index pages with the new locations
for (EntryLocation e : newLocations) {
- if (log.isDebugEnabled()) {
- log.debug("Update location - ledger: {} -- entry: {}", e.ledger, e.entry);
- }
+ log.debug()
+ .attr("ledgerId", e.ledger)
+ .attr("entryId", e.entry)
+ .log("Update location");
addLocation(batch, e.ledger, e.entry, e.location);
}
@@ -224,14 +225,12 @@ public void removeOffsetFromDeletedLedgers() throws IOException {
LongPairWrapper firstKeyWrapper = LongPairWrapper.get(-1, -1);
LongPairWrapper lastKeyWrapper = LongPairWrapper.get(-1, -1);
- log.info("Deleting indexes for ledgers: {}", ledgersToDelete);
+ log.info().attr("ledgers", ledgersToDelete).log("Deleting indexes for ledgers");
long startTime = System.nanoTime();
try (Batch batch = locationsDb.newBatch()) {
for (long ledgerId : ledgersToDelete) {
- if (log.isDebugEnabled()) {
- log.debug("Deleting indexes from ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Deleting indexes from ledger");
firstKeyWrapper.set(ledgerId, 0);
lastKeyWrapper.set(ledgerId, Long.MAX_VALUE);
@@ -248,9 +247,9 @@ public void removeOffsetFromDeletedLedgers() throws IOException {
lastKeyWrapper.recycle();
}
- log.info("Deleted indexes from {} ledgers in {} seconds", ledgersToDelete.size(),
- TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) / 1000.0);
+ log.info().attr("count", ledgersToDelete.size())
+ .attr("durationSeconds", TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) / 1000.0)
+ .log("Deleted indexes from ledgers");
}
- private static final Logger log = LoggerFactory.getLogger(EntryLocationIndex.class);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
index 1caea81be2c..97ce8d02df0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
@@ -39,6 +39,7 @@
import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.commons.lang3.StringUtils;
@@ -65,12 +66,11 @@
import org.rocksdb.Slice;
import org.rocksdb.WriteBatch;
import org.rocksdb.WriteOptions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* RocksDB based implementation of the KeyValueStorage.
*/
+@CustomLog
public class KeyValueStorageRocksDB implements KeyValueStorage {
static KeyValueStorageFactory factory = (defaultBasePath, subPath, dbConfigType, conf) ->
@@ -135,7 +135,7 @@ public KeyValueStorageRocksDB(String basePath, String subPath, DbConfigType dbCo
} else {
dbFilePath = conf.getDefaultRocksDBConf();
}
- log.info("Searching for a RocksDB configuration file in {}", dbFilePath);
+ log.info().attr("path", dbFilePath).log("Searching for a RocksDB configuration file");
if (StringUtils.isNotBlank(dbFilePath) && Paths.get(dbFilePath).toFile().exists()) {
log.info("Found a RocksDB configuration file and using it to initialize the RocksDB");
db = initializeRocksDBWithConfFile(basePath, subPath, dbConfigType, conf, readOnly, dbFilePath);
@@ -169,7 +169,10 @@ private RocksDB initializeRocksDBWithConfFile(String basePath, String subPath, D
if (!logPath.isEmpty()) {
Path logPathSetting = FileSystems.getDefault().getPath(logPath, subPath);
Files.createDirectories(logPathSetting);
- log.info("RocksDB<{}> log path: {}", subPath, logPathSetting);
+ log.info()
+ .attr("subPath", subPath)
+ .attr("logPath", logPathSetting)
+ .log("RocksDB log path");
dbOptions.setDbLogDir(logPathSetting.toString());
}
this.dbPath = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString();
@@ -253,7 +256,10 @@ private RocksDB initializeRocksDBWithBookieConf(String basePath, String subPath,
if (!logPath.isEmpty()) {
Path logPathSetting = FileSystems.getDefault().getPath(logPath, subPath);
Files.createDirectories(logPathSetting);
- log.info("RocksDB<{}> log path: {}", subPath, logPathSetting);
+ log.info()
+ .attr("subPath", subPath)
+ .attr("logPath", logPathSetting)
+ .log("RocksDB log path");
options.setDbLogDir(logPathSetting.toString());
}
this.dbPath = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString();
@@ -274,7 +280,7 @@ private RocksDB initializeRocksDBWithBookieConf(String basePath, String subPath,
options.setInfoLogLevel(InfoLogLevel.ERROR_LEVEL);
break;
default:
- log.warn("Unrecognized RockDB log level: {}", logLevel);
+ log.warn().attr("logLevel", logLevel).log("Unrecognized RockDB log level");
}
// Keep log files for 1month
@@ -406,16 +412,22 @@ public void compact() throws IOException {
final long start = System.currentTimeMillis();
final int oriRocksDBFileCount = db.getLiveFilesMetaData().size();
final long oriRocksDBSize = getRocksDBSize();
- log.info("Starting RocksDB {} compact, current RocksDB hold {} files and {} Bytes.",
- db.getName(), oriRocksDBFileCount, oriRocksDBSize);
+ log.info()
+ .attr("dbName", db.getName())
+ .attr("fileCount", oriRocksDBFileCount)
+ .attr("sizeBytes", oriRocksDBSize).log("Starting RocksDB compact");
db.compactRange();
final long end = System.currentTimeMillis();
final int rocksDBFileCount = db.getLiveFilesMetaData().size();
final long rocksDBSize = getRocksDBSize();
- log.info("RocksDB {} compact finished {} ms, space reduced {} Bytes, current hold {} files and {} Bytes.",
- db.getName(), end - start, oriRocksDBSize - rocksDBSize, rocksDBFileCount, rocksDBSize);
+ log.info()
+ .attr("dbName", db.getName())
+ .attr("durationMs", end - start)
+ .attr("spaceReducedBytes", oriRocksDBSize - rocksDBSize)
+ .attr("fileCount", rocksDBFileCount).attr("sizeBytes", rocksDBSize)
+ .log("RocksDB compact finished");
} catch (RocksDBException e) {
throw new IOException("Error in RocksDB compact", e);
}
@@ -665,5 +677,4 @@ RocksObject getOptions() {
return options;
}
- private static final Logger log = LoggerFactory.getLogger(KeyValueStorageRocksDB.class);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
index 5ca9f1ffd3e..7162c0c6c23 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
@@ -34,6 +34,7 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.Bookie;
import org.apache.bookkeeper.bookie.BookieException;
import org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorageDataFormats.LedgerData;
@@ -42,14 +43,13 @@
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Maintains an index for the ledgers metadata.
*
* The key is the ledgerId and the value is the {@link LedgerData} content.
*/
+@CustomLog
public class LedgerMetadataIndex implements Closeable {
// Non-ledger data should have negative ID
private static final long STORAGE_FLAGS = -0xeefd;
@@ -111,9 +111,7 @@ public void close() throws IOException {
public LedgerData get(long ledgerId) throws IOException {
LedgerData ledgerData = ledgers.get(ledgerId);
if (ledgerData == null) {
- if (log.isDebugEnabled()) {
- log.debug("Ledger not found {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Ledger not found");
throw new Bookie.NoLedgerException(ledgerId);
}
@@ -127,9 +125,7 @@ public void set(long ledgerId, LedgerData ledgerData) throws IOException {
lock.lock();
try {
if (ledgers.put(ledgerId, ledgerData) == null) {
- if (log.isDebugEnabled()) {
- log.debug("Added new ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Added new ledger");
ledgersCount.incrementAndGet();
}
@@ -145,9 +141,7 @@ public void delete(long ledgerId) throws IOException {
lock.lock();
try {
if (ledgers.remove(ledgerId) != null) {
- if (log.isDebugEnabled()) {
- log.debug("Removed ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Removed ledger");
ledgersCount.decrementAndGet();
}
@@ -184,12 +178,10 @@ public boolean setFenced(long ledgerId) throws IOException {
if (ledgers.put(ledgerId, newLedgerData) == null) {
// Ledger had been deleted
- if (log.isDebugEnabled()) {
- log.debug("Re-inserted fenced ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Re-inserted fenced ledger");
ledgersCount.incrementAndGet();
- } else if (log.isDebugEnabled()) {
- log.debug("Set fenced ledger {}", ledgerId);
+ } else {
+ log.debug().attr("ledgerId", ledgerId).log("Set fenced ledger");
}
pendingLedgersUpdates.add(new SimpleEntry(ledgerId, newLedgerData));
@@ -213,12 +205,10 @@ public boolean setLimbo(long ledgerId) throws IOException {
if (ledgers.put(ledgerId, newLedgerData) == null) {
// Ledger had been deleted
- if (log.isDebugEnabled()) {
- log.debug("Re-inserted limbo ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Re-inserted limbo ledger");
ledgersCount.incrementAndGet();
- } else if (log.isDebugEnabled()) {
- log.debug("Set limbo ledger {}", ledgerId);
+ } else {
+ log.debug().attr("ledgerId", ledgerId).log("Set limbo ledger");
}
pendingLedgersUpdates.add(new SimpleEntry(ledgerId, newLedgerData));
@@ -242,12 +232,10 @@ public boolean clearLimbo(long ledgerId) throws IOException {
if (ledgers.put(ledgerId, newLedgerData) == null) {
// Ledger had been deleted
- if (log.isDebugEnabled()) {
- log.debug("Re-inserted limbo ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Re-inserted limbo ledger");
ledgersCount.incrementAndGet();
- } else if (log.isDebugEnabled()) {
- log.debug("Set limbo ledger {}", ledgerId);
+ } else {
+ log.debug().attr("ledgerId", ledgerId).log("Set limbo ledger");
}
pendingLedgersUpdates.add(new SimpleEntry(ledgerId, newLedgerData));
@@ -268,19 +256,18 @@ public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException {
// New ledger inserted
ledgerData = LedgerData.newBuilder().setExists(true).setFenced(false)
.setMasterKey(ByteString.copyFrom(masterKey)).build();
- if (log.isDebugEnabled()) {
- log.debug("Inserting new ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Inserting new ledger");
} else {
byte[] storedMasterKey = ledgerData.getMasterKey().toByteArray();
if (ArrayUtil.isArrayAllZeros(storedMasterKey)) {
// update master key of the ledger
ledgerData = LedgerData.newBuilder(ledgerData).setMasterKey(ByteString.copyFrom(masterKey)).build();
- if (log.isDebugEnabled()) {
- log.debug("Replace old master key {} with new master key {}", storedMasterKey, masterKey);
- }
+ log.debug()
+ .attr("storedMasterKey", storedMasterKey)
+ .attr("newMasterKey", masterKey)
+ .log("Replace old master key with new master key");
} else if (!Arrays.equals(storedMasterKey, masterKey) && !ArrayUtil.isArrayAllZeros(masterKey)) {
- log.warn("Ledger {} masterKey in db can only be set once.", ledgerId);
+ log.warn().attr("ledgerId", ledgerId).log("Ledger masterKey in db can only be set once");
throw new IOException(BookieException.create(BookieException.Code.IllegalOpException));
}
}
@@ -316,9 +303,7 @@ public void flush() throws IOException {
++updatedLedgers;
}
- if (log.isDebugEnabled()) {
- log.debug("Persisting updates to {} ledgers", updatedLedgers);
- }
+ log.debug().attr("updatedLedgers", updatedLedgers).log("Persisting updates to ledgers");
ledgersDb.sync();
} finally {
@@ -342,9 +327,7 @@ public void removeDeletedLedgers() throws IOException {
pendingDeletedLedgers.remove(ledgerId);
}
- if (log.isDebugEnabled()) {
- log.debug("Persisting deletes of ledgers {}", deletedLedgers);
- }
+ log.debug().attr("deletedLedgers", deletedLedgers).log("Persisting deletes of ledgers");
ledgersDb.sync();
} finally {
@@ -402,8 +385,6 @@ boolean setStorageStateFlags(int expected, int newFlags) throws IOException {
return false;
}
- private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndex.class);
-
void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
LedgerData ledgerData = ledgers.get(ledgerId);
if (ledgerData != null) {
@@ -414,8 +395,8 @@ void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
// Ledger had been deleted
ledgersCount.incrementAndGet();
return;
- } else if (log.isDebugEnabled()) {
- log.debug("Set explicitLac on ledger {}", ledgerId);
+ } else {
+ log.debug().attr("ledgerId", ledgerId).log("Set explicitLac on ledger");
}
pendingLedgersUpdates.add(new SimpleEntry(ledgerId, newLedgerData));
} else {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
index 0160f11f577..8a7df8f5652 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
@@ -27,18 +27,17 @@
import java.util.Base64;
import java.util.Map;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.commons.lang3.time.DurationFormatUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Scan the ledgers index to make sure it is readable.
*/
+@CustomLog
public class LedgersIndexCheckOp {
- private static final Logger LOG = LoggerFactory.getLogger(LedgersIndexCheckOp.class);
private final ServerConfiguration conf;
private final boolean verbose;
@@ -64,8 +63,8 @@ public boolean initiate() throws IOException {
String iBasePath = BookieImpl.getCurrentDirectory(indexDir).toString();
Path indexCurrentPath = FileSystems.getDefault().getPath(iBasePath, LedgersSubPath);
- LOG.info("Loading ledgers index from {}", indexCurrentPath);
- LOG.info("Starting index scan");
+ log.info().attr("indexPath", indexCurrentPath).log("Loading ledgers index");
+ log.info("Starting index scan");
try {
KeyValueStorage index = new KeyValueStorageRocksDB(iBasePath, LedgersSubPath,
@@ -81,33 +80,34 @@ public boolean initiate() throws IOException {
DbLedgerStorageDataFormats.LedgerData ledgerData =
DbLedgerStorageDataFormats.LedgerData.parseFrom(entry.getValue());
if (verbose) {
- LOG.info(
- "Scanned: {}, ledger: {}, exists: {}, isFenced: {}, masterKey: {}, explicitLAC: {}",
- ctr,
- ledgerId,
- (ledgerData.hasExists() ? ledgerData.getExists() : "-"),
- (ledgerData.hasFenced() ? ledgerData.getFenced() : "-"),
- (ledgerData.hasMasterKey()
+ log.info()
+ .attr("scanned", ctr)
+ .attr("ledgerId", ledgerId)
+ .attr("exists", (ledgerData.hasExists() ? ledgerData.getExists() : "-"))
+ .attr("isFenced", (ledgerData.hasFenced() ? ledgerData.getFenced() : "-"))
+ .attr("masterKey", (ledgerData.hasMasterKey()
? Base64.getEncoder()
.encodeToString(ledgerData.getMasterKey().toByteArray())
- : "-"),
- (ledgerData.hasExplicitLac() ? ledgerData.getExplicitLac() : "-"));
+ : "-"))
+ .attr("explicitLAC", (ledgerData.hasExplicitLac()
+ ? ledgerData.getExplicitLac() : "-"))
+ .log("Scanned ledger");
} else if (ctr % 100 == 0) {
- LOG.info("Scanned {} ledgers", ctr);
+ log.info().attr("count", ctr).log("Scanned ledgers");
}
}
} finally {
iterator.close();
}
- LOG.info("Scanned {} ledgers", ctr);
+ log.info().attr("count", ctr).log("Scanned ledgers");
} catch (Throwable t) {
- LOG.error("Index scan has failed with error", t);
+ log.error().exception(t).log("Index scan has failed with error");
return false;
}
}
- LOG.info("Index scan has completed successfully. Total time: {}",
- DurationFormatUtils.formatDurationHMS(
- TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)));
+ log.info().attr("totalTime", DurationFormatUtils.formatDurationHMS(
+ TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)))
+ .log("Index scan has completed successfully");
return true;
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
index 2725897e804..fc9f94559a8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
@@ -36,6 +36,7 @@
import java.util.HashSet;
import java.util.List;
import java.util.Set;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.DefaultEntryLogger;
import org.apache.bookkeeper.bookie.Journal;
@@ -45,8 +46,6 @@
import org.apache.bookkeeper.conf.ServerConfiguration;
import org.apache.bookkeeper.util.BookKeeperConstants;
import org.apache.bookkeeper.util.DiskChecker;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Scan all entries in the journal and entry log files then rebuilds the ledgers index.
@@ -59,8 +58,8 @@
* are overwritten and we cannot use the password from metadata, and cannot know 100%
* for sure how a digest for the password was generated.
*/
+@CustomLog
public class LedgersIndexRebuildOp {
- private static final Logger LOG = LoggerFactory.getLogger(LedgersIndexRebuildOp.class);
private final ServerConfiguration conf;
private final boolean verbose;
@@ -73,13 +72,13 @@ public LedgersIndexRebuildOp(ServerConfiguration conf, boolean verbose) {
@SuppressFBWarnings("RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE")
public boolean initiate() {
- LOG.info("Starting ledger index rebuilding");
+ log.info("Starting ledger index rebuilding");
File[] indexDirs = conf.getIndexDirs();
if (indexDirs == null) {
indexDirs = conf.getLedgerDirs();
}
if (indexDirs.length != conf.getLedgerDirs().length) {
- LOG.error("ledger and index dirs size not matched");
+ log.error("ledger and index dirs size not matched");
return false;
}
@@ -93,7 +92,7 @@ public boolean initiate() {
Path indexTempPath = FileSystems.getDefault().getPath(indexBasePath, tempLedgersSubPath);
Path indexCurrentPath = FileSystems.getDefault().getPath(indexBasePath, LedgersSubPath);
- LOG.info("Starting scan phase (scans journal and entry log files)");
+ log.info("Starting scan phase (scans journal and entry log files)");
try {
Set ledgers = new HashSet<>();
@@ -102,12 +101,12 @@ public boolean initiate() {
lDirs[0] = ledgerDir;
scanEntryLogFiles(ledgers, lDirs);
- LOG.info("Scan complete, found {} ledgers. "
- + "Starting to build a new ledgers index", ledgers.size());
+ log.info().attr("count", ledgers.size())
+ .log("Scan complete. Starting to build a new ledgers index");
try (KeyValueStorage newIndex = KeyValueStorageRocksDB.factory.newKeyValueStorage(
indexBasePath, tempLedgersSubPath, DbConfigType.Default, conf)) {
- LOG.info("Created ledgers index at temp location {}", indexTempPath);
+ log.info().attr("tempPath", indexTempPath).log("Created ledgers index at temp location");
for (Long ledgerId : ledgers) {
DbLedgerStorageDataFormats.LedgerData ledgerData =
@@ -124,7 +123,7 @@ public boolean initiate() {
newIndex.sync();
}
} catch (Throwable t) {
- LOG.error("Error during rebuild, the original index remains unchanged", t);
+ log.error().exception(t).log("Error during rebuild, the original index remains unchanged");
delete(indexTempPath);
return false;
}
@@ -133,16 +132,21 @@ public boolean initiate() {
try {
Path prevPath = FileSystems.getDefault().getPath(indexBasePath,
LedgersSubPath + ".PREV-" + timestamp);
- LOG.info("Moving original index from original location: {} up to back-up location: {}",
- indexCurrentPath, prevPath);
+ log.info()
+ .attr("from", indexCurrentPath)
+ .attr("to", prevPath)
+ .log("Moving original index to back-up location");
Files.move(indexCurrentPath, prevPath);
- LOG.info("Moving rebuilt index from: {} to: {}", indexTempPath, indexCurrentPath);
+ log.info()
+ .attr("from", indexTempPath)
+ .attr("to", indexCurrentPath)
+ .log("Moving rebuilt index");
Files.move(indexTempPath, indexCurrentPath);
- LOG.info("Original index has been replaced with the new index. "
- + "The original index has been moved to {}", prevPath);
+ log.info().attr("backupPath", prevPath)
+ .log("Original index has been replaced with the new index");
} catch (IOException e) {
- LOG.error("Could not replace original index with rebuilt index. "
- + "To return to the original state, ensure the original index is in its original location", e);
+ log.error().exception(e).log("Could not replace original index with rebuilt index. "
+ + "To return to the original state, ensure the original index is in its original location");
return false;
}
}
@@ -157,7 +161,7 @@ private void scanEntryLogFiles(Set ledgers, File[] lDirs) throws IOExcepti
int totalEntryLogs = entryLogs.size();
int completedEntryLogs = 0;
- LOG.info("Scanning {} entry logs", totalEntryLogs);
+ log.info().attr("count", totalEntryLogs).log("Scanning entry logs");
for (long entryLogId : entryLogs) {
entryLogger.scanEntryLog(entryLogId, new EntryLogScanner() {
@@ -165,7 +169,7 @@ private void scanEntryLogFiles(Set ledgers, File[] lDirs) throws IOExcepti
public void process(long ledgerId, long offset, ByteBuf entry) throws IOException {
if (ledgers.add(ledgerId)) {
if (verbose) {
- LOG.info("Found ledger {} in entry log", ledgerId);
+ log.info().attr("ledgerId", ledgerId).log("Found ledger in entry log");
}
}
}
@@ -177,8 +181,9 @@ public boolean accept(long ledgerId) {
});
++completedEntryLogs;
- LOG.info("Completed scanning of log {}.log -- {} / {}", Long.toHexString(entryLogId), completedEntryLogs,
- totalEntryLogs);
+ log.info().attr("entryLogId", Long.toHexString(entryLogId))
+ .attr("completed", completedEntryLogs).attr("total", totalEntryLogs)
+ .log("Completed scanning of entry log");
}
}
@@ -211,7 +216,10 @@ private List getJournals(ServerConfiguration conf) throws IOException {
}
private void scanJournal(Journal journal, long journalId, Set ledgers) throws IOException {
- LOG.info("Scanning journal " + journalId + " (" + Long.toHexString(journalId) + ".txn)");
+ log.info()
+ .attr("journalId", journalId)
+ .attr("journalFile", Long.toHexString(journalId) + ".txn")
+ .log("Scanning journal");
journal.scanJournal(journalId, 0L, new Journal.JournalScanner() {
@Override
public void process(int journalVersion, long offset, ByteBuffer entry) {
@@ -219,7 +227,7 @@ public void process(int journalVersion, long offset, ByteBuffer entry) {
long ledgerId = buf.readLong();
if (ledgers.add(ledgerId) && verbose) {
- LOG.info("Found ledger {} in journal", ledgerId);
+ log.info().attr("ledgerId", ledgerId).log("Found ledger in journal");
}
}
}, false);
@@ -229,7 +237,10 @@ private void delete(Path path) {
try {
Files.delete(path);
} catch (IOException e) {
- LOG.warn("Unable to delete {}", path.toAbsolutePath(), e);
+ log.warn()
+ .attr("path", path.toAbsolutePath())
+ .exception(e)
+ .log("Unable to delete");
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
index 06afd9a823f..c0aa3161376 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
@@ -33,6 +33,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookieImpl;
import org.apache.bookkeeper.bookie.DefaultEntryLogger;
import org.apache.bookkeeper.bookie.LedgerDirsManager;
@@ -42,12 +43,11 @@
import org.apache.bookkeeper.stats.NullStatsLogger;
import org.apache.bookkeeper.util.DiskChecker;
import org.apache.commons.lang3.time.DurationFormatUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Scan all entries in the entry log and rebuild the locations index.
*/
+@CustomLog
public class LocationsIndexRebuildOp {
private final ServerConfiguration conf;
@@ -58,7 +58,7 @@ public LocationsIndexRebuildOp(ServerConfiguration conf) {
private static final int BATCH_COMMIT_SIZE = 10_000;
public void initiate() throws IOException {
- LOG.info("Starting locations index rebuilding");
+ log.info("Starting locations index rebuilding");
File[] indexDirs = conf.getIndexDirs();
if (indexDirs == null) {
indexDirs = conf.getLedgerDirs();
@@ -77,7 +77,7 @@ public void initiate() throws IOException {
Path backupPath = FileSystems.getDefault().getPath(iBasePath, "locations.BACKUP-" + timestamp);
Files.move(indexCurrentPath, backupPath);
- LOG.info("Created locations index backup at {}", backupPath);
+ log.info().attr("backupPath", backupPath).log("Created locations index backup");
File[] lDirs = new File[1];
lDirs[0] = ledgerDir;
@@ -86,14 +86,14 @@ public void initiate() throws IOException {
Set entryLogs = entryLogger.getEntryLogsSet();
Set activeLedgers = getActiveLedgers(conf, KeyValueStorageRocksDB.factory, iBasePath);
- LOG.info("Found {} active ledgers in ledger manager", activeLedgers.size());
+ log.info().attr("count", activeLedgers.size()).log("Found active ledgers in ledger manager");
KeyValueStorage newIndex = KeyValueStorageRocksDB.factory.newKeyValueStorage(iBasePath, "locations",
DbConfigType.Default, conf);
int totalEntryLogs = entryLogs.size();
int completedEntryLogs = 0;
- LOG.info("Scanning {} entry logs", totalEntryLogs);
+ log.info().attr("totalEntryLogs", totalEntryLogs).log("Scanning entry logs");
AtomicReference batch = new AtomicReference<>(newIndex.newBatch());
AtomicInteger count = new AtomicInteger();
@@ -106,10 +106,11 @@ public void process(long ledgerId, long offset, ByteBuf entry) throws IOExceptio
// Actual location indexed is pointing past the entry size
long location = (entryLogId << 32L) | (offset + 4);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Rebuilding {}:{} at location {} / {}", ledgerId, entryId, location >> 32,
- location & (Integer.MAX_VALUE - 1));
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("logId", location >> 32).attr("offset", location & (Integer.MAX_VALUE - 1))
+ .log("Rebuilding entry location");
// Update the ledger index page
LongPairWrapper key = LongPairWrapper.get(ledgerId, entryId);
@@ -138,8 +139,9 @@ public boolean accept(long ledgerId) {
});
++completedEntryLogs;
- LOG.info("Completed scanning of log {}.log -- {} / {}", Long.toHexString(entryLogId),
- completedEntryLogs, totalEntryLogs);
+ log.info().attr("entryLogId", Long.toHexString(entryLogId))
+ .attr("completed", completedEntryLogs).attr("total", totalEntryLogs)
+ .log("Completed scanning of entry log");
}
batch.get().flush();
@@ -148,9 +150,9 @@ public boolean accept(long ledgerId) {
newIndex.sync();
newIndex.close();
}
- LOG.info("Rebuilding index is done. Total time: {}",
- DurationFormatUtils.formatDurationHMS(
- TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)));
+ log.info().attr("totalTime", DurationFormatUtils.formatDurationHMS(
+ TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)))
+ .log("Rebuilding index is done");
}
private Set getActiveLedgers(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath)
@@ -165,5 +167,4 @@ private Set getActiveLedgers(ServerConfiguration conf, KeyValueStorageFact
return activeLedgers;
}
- private static final Logger LOG = LoggerFactory.getLogger(LocationsIndexRebuildOp.class);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
index 1d850c456c6..94b608ef137 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
@@ -31,10 +31,9 @@
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Read cache implementation.
@@ -46,8 +45,8 @@
* is cleared and rotated to make space for new entries to be added to
* the read cache.
*/
+@CustomLog
public class ReadCache implements Closeable {
- private static final Logger log = LoggerFactory.getLogger(ReadCache.class);
private static final int DEFAULT_MAX_SEGMENT_SIZE = 1 * 1024 * 1024 * 1024;
@@ -97,7 +96,10 @@ public void put(long ledgerId, long entryId, ByteBuf entry) {
try {
if (entrySize > segmentSize) {
- log.warn("entrySize {} > segmentSize {}, skip update read cache!", entrySize, segmentSize);
+ log.warn()
+ .attr("entrySize", entrySize)
+ .attr("segmentSize", segmentSize)
+ .log("entrySize > segmentSize, skip update read cache!");
return;
}
int offset = currentSegmentOffset.getAndAdd(alignedSize);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
index 774d10c158f..118d9692ddf 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
@@ -49,6 +49,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.ReentrantLock;
import java.util.concurrent.locks.StampedLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.Bookie;
import org.apache.bookkeeper.bookie.Bookie.NoEntryException;
import org.apache.bookkeeper.bookie.BookieException;
@@ -81,8 +82,6 @@
import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.mutable.MutableLong;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Single directory implementation of LedgerStorage that uses RocksDB to keep the indexes for entries stored in
@@ -90,6 +89,7 @@
*
* This is meant only to be used from {@link DbLedgerStorage}.
*/
+@CustomLog
public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage {
private final EntryLogger entryLogger;
@@ -166,13 +166,13 @@ public SingleDirectoryDbLedgerStorage(ServerConfiguration conf, LedgerManager le
String indexBaseDir = ledgerBaseDir;
if (CollectionUtils.isEmpty(indexDirsManager.getAllLedgerDirs())
|| ledgerBaseDir.equals(indexDirsManager.getAllLedgerDirs().get(0).getPath())) {
- log.info("indexDir is equals ledgerBaseDir, creating single directory db ledger storage on {}",
- indexBaseDir);
+ log.info().attr("indexBaseDir", indexBaseDir)
+ .log("indexDir is equals ledgerBaseDir, creating single directory db ledger storage");
} else {
// if indexDir is specified, set new value
indexBaseDir = indexDirsManager.getAllLedgerDirs().get(0).getPath();
- log.info("indexDir is specified a separate dir, creating single directory db ledger storage on {}",
- indexBaseDir);
+ log.info().attr("indexBaseDir", indexBaseDir)
+ .log("indexDir is specified a separate dir, creating single directory db ledger storage");
}
this.indexBaseDir = indexBaseDir;
@@ -316,8 +316,8 @@ public boolean isMinorGcSuspended() {
public void entryLocationCompact() {
if (entryLocationIndex.isCompacting()) {
// RocksDB already running compact.
- log.info("Compacting directory {}, skipping this entryLocationCompaction this time.",
- entryLocationIndex.getEntryLocationDBPath());
+ log.info().attr("directory", entryLocationIndex.getEntryLocationDBPath())
+ .log("Compacting directory, skipping this entryLocationCompaction this time");
return;
}
cleanupExecutor.execute(() -> {
@@ -327,7 +327,7 @@ public void entryLocationCompact() {
log.info("Trigger entry location index RocksDB compact.");
entryLocationIndex.compact();
} catch (Throwable t) {
- log.warn("Failed to trigger entry location index RocksDB compact", t);
+ log.warn().exception(t).log("Failed to trigger entry location index RocksDB compact");
}
});
}
@@ -362,7 +362,7 @@ public void shutdown() throws InterruptedException {
executor.shutdown();
} catch (IOException e) {
- log.error("Error closing db storage", e);
+ log.error().exception(e).log("Error closing db storage");
}
}
@@ -370,9 +370,10 @@ public void shutdown() throws InterruptedException {
public boolean ledgerExists(long ledgerId) throws IOException {
try {
LedgerData ledgerData = ledgerIndex.get(ledgerId);
- if (log.isDebugEnabled()) {
- log.debug("Ledger exists. ledger: {} : {}", ledgerId, ledgerData.getExists());
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("exists", ledgerData.getExists())
+ .log("Ledger exists");
return ledgerData.getExists();
} catch (Bookie.NoLedgerException nle) {
// ledger does not exist
@@ -427,9 +428,10 @@ public boolean entryExists(long ledgerId, long entryId) throws IOException, Book
public boolean isFenced(long ledgerId) throws IOException, BookieException {
boolean isFenced = ledgerIndex.get(ledgerId).getFenced();
- if (log.isDebugEnabled()) {
- log.debug("ledger: {}, isFenced: {}.", ledgerId, isFenced);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("isFenced", isFenced)
+ .log("isFenced check");
// Only a negative result while in limbo equates to unknown
if (!isFenced) {
@@ -441,9 +443,7 @@ public boolean isFenced(long ledgerId) throws IOException, BookieException {
@Override
public boolean setFenced(long ledgerId) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("Set fenced. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Set fenced");
boolean changed = ledgerIndex.setFenced(ledgerId);
if (changed) {
// notify all the watchers if a ledger is fenced
@@ -457,17 +457,13 @@ public boolean setFenced(long ledgerId) throws IOException {
@Override
public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("Set master key. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Set master key");
ledgerIndex.setMasterKey(ledgerId, masterKey);
}
@Override
public byte[] readMasterKey(long ledgerId) throws IOException, BookieException {
- if (log.isDebugEnabled()) {
- log.debug("Read master key. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Read master key");
return ledgerIndex.get(ledgerId).getMasterKey().toByteArray();
}
@@ -479,9 +475,11 @@ public long addEntry(ByteBuf entry) throws IOException, BookieException {
long entryId = entry.getLong(entry.readerIndex() + 8);
long lac = entry.getLong(entry.readerIndex() + 16);
- if (log.isDebugEnabled()) {
- log.debug("Add entry. {}@{}, lac = {}", ledgerId, entryId, lac);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("lac", lac)
+ .log("Add entry");
// First we try to do an optimistic locking to get access to the current write cache.
// This is based on the fact that the write cache is only being rotated (swapped) every 1 minute. During the
@@ -535,7 +533,7 @@ private void triggerFlushAndAddEntry(long ledgerId, long entryId, ByteBuf entry)
try {
flush();
} catch (IOException e) {
- log.error("Error during flush", e);
+ log.error().exception(e).log("Error during flush");
} finally {
flushExecutorTime.addLatency(MathUtils.elapsedNanos(startTime), TimeUnit.NANOSECONDS);
}
@@ -582,9 +580,10 @@ public ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieE
}
private ByteBuf doGetEntry(long ledgerId, long entryId) throws IOException, BookieException {
- if (log.isDebugEnabled()) {
- log.debug("Get Entry: {}@{}", ledgerId, entryId);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .log("Get Entry");
if (entryId == BookieProtocol.LAST_ADD_CONFIRMED) {
return getLastEntry(ledgerId);
@@ -702,9 +701,10 @@ private void fillReadAheadCache(long originalLedgerId, long firstEntryId, long f
}
}
} catch (Exception e) {
- if (log.isDebugEnabled()) {
- log.debug("Exception during read ahead for ledger: {}: e", originalLedgerId, e);
- }
+ log.debug()
+ .attr("ledgerId", originalLedgerId)
+ .exception(e)
+ .log("Exception during read ahead for ledger");
} finally {
dbLedgerStorageStats.getReadAheadBatchCountStats().registerSuccessfulValue(count);
dbLedgerStorageStats.getReadAheadBatchSizeStats().registerSuccessfulValue(size);
@@ -732,15 +732,11 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException, BookieException {
// First try to read from the write cache of recent entries
ByteBuf entry = writeCache.getLastEntry(ledgerId);
if (entry != null) {
- if (log.isDebugEnabled()) {
- long foundLedgerId = entry.readLong(); // ledgerId
- long entryId = entry.readLong();
- entry.resetReaderIndex();
- if (log.isDebugEnabled()) {
- log.debug("Found last entry for ledger {} in write cache: {}@{}", ledgerId, foundLedgerId,
- entryId);
- }
- }
+ final ByteBuf found = entry;
+ log.debug(e -> e.attr("ledgerId", ledgerId)
+ .attr("foundLedgerId", found.getLong(0))
+ .attr("entryId", found.getLong(8))
+ .log("Found last entry for ledger in write cache"));
dbLedgerStorageStats.getWriteCacheHitCounter().inc();
return entry;
@@ -749,14 +745,11 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException, BookieException {
// If there's a flush going on, the entry might be in the flush buffer
entry = writeCacheBeingFlushed.getLastEntry(ledgerId);
if (entry != null) {
- if (log.isDebugEnabled()) {
- entry.readLong(); // ledgerId
- long entryId = entry.readLong();
- entry.resetReaderIndex();
- if (log.isDebugEnabled()) {
- log.debug("Found last entry for ledger {} in write cache being flushed: {}", ledgerId, entryId);
- }
- }
+ final ByteBuf found = entry;
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", () -> found.getLong(8))
+ .log("Found last entry for ledger in write cache being flushed");
dbLedgerStorageStats.getWriteCacheHitCounter().inc();
return entry;
@@ -770,9 +763,10 @@ public ByteBuf getLastEntry(long ledgerId) throws IOException, BookieException {
// Search the last entry in storage
long locationIndexStartNano = MathUtils.nowInNano();
long lastEntryId = entryLocationIndex.getLastEntryInLedger(ledgerId);
- if (log.isDebugEnabled()) {
- log.debug("Found last entry for ledger {} in db: {}", ledgerId, lastEntryId);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("lastEntryId", lastEntryId)
+ .log("Found last entry for ledger in db");
long entryLocation = entryLocationIndex.getLocation(ledgerId, lastEntryId);
dbLedgerStorageStats.getReadFromLocationIndexTime().addLatency(
@@ -822,10 +816,9 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
swapWriteCache();
long sizeToFlush = writeCacheBeingFlushed.size();
- if (log.isDebugEnabled()) {
- log.debug("Flushing entries. count: {} -- size {} Mb", writeCacheBeingFlushed.count(),
- sizeToFlush / 1024.0 / 1024);
- }
+ log.debug(e -> e.attr("count", writeCacheBeingFlushed.count())
+ .attr("sizeMb", sizeToFlush / 1024.0 / 1024)
+ .log("Flushing entries"));
// Write all the pending entries into the entry logger and collect the offset
// position for each entry
@@ -844,10 +837,9 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
batch.flush();
recordSuccessfulEvent(dbLedgerStorageStats.getFlushLocationIndexStats(), batchFlushStartTime);
- if (log.isDebugEnabled()) {
- log.debug("DB batch flushed time : {} s",
- MathUtils.elapsedNanos(batchFlushStartTime) / (double) TimeUnit.SECONDS.toNanos(1));
- }
+ log.debug().attr("durationSeconds",
+ () -> MathUtils.elapsedNanos(batchFlushStartTime) / (double) TimeUnit.SECONDS.toNanos(1))
+ .log("DB batch flushed");
}
long ledgerIndexStartTime = MathUtils.nowInNano();
@@ -862,9 +854,10 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
double flushTimeSeconds = MathUtils.elapsedNanos(startTime) / (double) TimeUnit.SECONDS.toNanos(1);
double flushThroughput = sizeToFlush / 1024.0 / 1024.0 / flushTimeSeconds;
- if (log.isDebugEnabled()) {
- log.debug("Flushing done time {} s -- Written {} MB/s", flushTimeSeconds, flushThroughput);
- }
+ log.debug()
+ .attr("durationSeconds", flushTimeSeconds)
+ .attr("throughputMBs", flushThroughput)
+ .log("Flushing done");
recordSuccessfulEvent(dbLedgerStorageStats.getFlushStats(), startTime);
dbLedgerStorageStats.getFlushSizeStats().registerSuccessfulValue(sizeToFlush);
@@ -878,14 +871,12 @@ public void checkpoint(Checkpoint checkpoint) throws IOException {
// There can only be one single cleanup task running because the cleanupExecutor
// is single-threaded
try {
- if (log.isDebugEnabled()) {
- log.debug("Removing deleted ledgers from db indexes");
- }
+ log.debug("Removing deleted ledgers from db indexes");
entryLocationIndex.removeOffsetFromDeletedLedgers();
ledgerIndex.removeDeletedLedgers();
} catch (Throwable t) {
- log.warn("Failed to cleanup db indexes", t);
+ log.warn().exception(t).log("Failed to cleanup db indexes");
}
});
@@ -928,9 +919,7 @@ public void flush() throws IOException {
@Override
public void deleteLedger(long ledgerId) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("Deleting ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Deleting ledger");
// Delete entries from this ledger that are still in the write cache
long stamp = writeCacheRotationLock.readLock();
@@ -1036,26 +1025,18 @@ public void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
@Override
public ByteBuf getExplicitLac(long ledgerId) throws IOException, BookieException {
throwIfLimbo(ledgerId);
- if (log.isDebugEnabled()) {
- log.debug("getExplicitLac ledger {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("getExplicitLac");
TransientLedgerInfo ledgerInfo = getOrAddLedgerInfo(ledgerId);
if (ledgerInfo.getExplicitLac() != null) {
- if (log.isDebugEnabled()) {
- log.debug("getExplicitLac ledger {} returned from TransientLedgerInfo", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("getExplicitLac returned from TransientLedgerInfo");
return ledgerInfo.getExplicitLac();
}
LedgerData ledgerData = ledgerIndex.get(ledgerId);
if (!ledgerData.hasExplicitLac()) {
- if (log.isDebugEnabled()) {
- log.debug("getExplicitLac ledger {} missing from LedgerData", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("getExplicitLac missing from LedgerData");
return null;
}
- if (log.isDebugEnabled()) {
- log.debug("getExplicitLac ledger {} returned from LedgerData", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("getExplicitLac returned from LedgerData");
ByteString persistedLac = ledgerData.getExplicitLac();
ledgerInfo.setExplicitLac(Unpooled.wrappedBuffer(persistedLac.toByteArray()));
return ledgerInfo.getExplicitLac();
@@ -1146,8 +1127,6 @@ public interface LedgerLoggerProcessor {
void process(long entryId, long entryLogId, long position);
}
- private static final Logger log = LoggerFactory.getLogger(SingleDirectoryDbLedgerStorage.class);
-
@Override
public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException {
throw new UnsupportedOperationException(
@@ -1231,33 +1210,25 @@ public void diskJustWritable(File disk) {
@Override
public void setLimboState(long ledgerId) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("setLimboState. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("setLimboState");
ledgerIndex.setLimbo(ledgerId);
}
@Override
public boolean hasLimboState(long ledgerId) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("hasLimboState. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("hasLimboState");
return ledgerIndex.get(ledgerId).getLimbo();
}
@Override
public void clearLimboState(long ledgerId) throws IOException {
- if (log.isDebugEnabled()) {
- log.debug("clearLimboState. ledger: {}", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("clearLimboState");
ledgerIndex.clearLimbo(ledgerId);
}
private void throwIfLimbo(long ledgerId) throws IOException, BookieException {
if (hasLimboState(ledgerId)) {
- if (log.isDebugEnabled()) {
- log.debug("Accessing ledger({}) in limbo state, throwing exception", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Accessing ledger in limbo state, throwing exception");
throw BookieException.create(BookieException.Code.DataUnknownException);
}
}
@@ -1294,8 +1265,10 @@ public void setStorageStateFlag(StorageState flag) throws IOException {
if (ledgerIndex.setStorageStateFlags(curFlags, newFlags)) {
return;
} else {
- log.info("Conflict updating storage state flags {} -> {}, retrying",
- curFlags, newFlags);
+ log.info()
+ .attr("curFlags", curFlags)
+ .attr("newFlags", newFlags)
+ .log("Conflict updating storage state flags, retrying");
}
}
}
@@ -1310,8 +1283,10 @@ public void clearStorageStateFlag(StorageState flag) throws IOException {
if (ledgerIndex.setStorageStateFlags(curFlags, newFlags)) {
return;
} else {
- log.info("Conflict updating storage state flags {} -> {}, retrying",
- curFlags, newFlags);
+ log.info()
+ .attr("curFlags", curFlags)
+ .attr("newFlags", newFlags)
+ .log("Conflict updating storage state flags, retrying");
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
index d152c6fc1de..fb1e42adb24 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
@@ -30,13 +30,12 @@
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.common.util.MathUtils;
import org.apache.bookkeeper.util.collections.ConcurrentLongHashSet;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap;
import org.apache.bookkeeper.util.collections.ConcurrentLongLongPairHashMap.LongPair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Write cache implementation.
@@ -50,6 +49,7 @@
*
There is the possibility to iterate through the stored entries in an ordered
* way, by (ledgerId, entry).
*/
+@CustomLog
public class WriteCache implements Closeable {
/**
@@ -243,16 +243,16 @@ public void forEach(EntryConsumer consumer) throws IOException {
sortedEntriesIdx += 4;
});
- if (log.isDebugEnabled()) {
- log.debug("iteration took {} ms", MathUtils.elapsedNanos(startTime) / 1e6);
- }
+ final long iterationStart = startTime;
+ log.debug().attr("durationMs", () -> MathUtils.elapsedNanos(iterationStart) / 1e6)
+ .log("iteration took");
startTime = MathUtils.nowInNano();
// Sort entries by (ledgerId, entryId) maintaining the 4 items groups
ArrayGroupSort.sort(sortedEntries, 0, sortedEntriesIdx);
- if (log.isDebugEnabled()) {
- log.debug("sorting {} ms", (MathUtils.elapsedNanos(startTime) / 1e6));
- }
+ final long sortStart = startTime;
+ log.debug().attr("durationMs", () -> MathUtils.elapsedNanos(sortStart) / 1e6)
+ .log("sorting");
startTime = MathUtils.nowInNano();
ByteBuf[] entrySegments = new ByteBuf[segmentsCount];
@@ -273,9 +273,9 @@ public void forEach(EntryConsumer consumer) throws IOException {
consumer.accept(ledgerId, entryId, entry);
}
- if (log.isDebugEnabled()) {
- log.debug("entry log adding {} ms", MathUtils.elapsedNanos(startTime) / 1e6);
- }
+ final long entryLogStart = startTime;
+ log.debug().attr("durationMs", () -> MathUtils.elapsedNanos(entryLogStart) / 1e6)
+ .log("entry log adding");
} finally {
sortedEntriesLock.unlock();
}
@@ -307,5 +307,4 @@ private static long alignToPowerOfTwo(long n) {
private long[] sortedEntries;
private int sortedEntriesIdx;
- private static final Logger log = LoggerFactory.getLogger(WriteCache.class);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BatchedReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BatchedReadOp.java
index 97a284d04ec..774d57ed92b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BatchedReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BatchedReadOp.java
@@ -25,6 +25,7 @@
import java.util.BitSet;
import java.util.List;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.api.LedgerEntry;
import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
@@ -34,13 +35,10 @@
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.BatchedReadEntryCallback;
import org.apache.bookkeeper.proto.checksum.DigestManager;
import org.apache.bookkeeper.util.ByteBufList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+@CustomLog
public class BatchedReadOp extends ReadOpBase implements BatchedReadEntryCallback {
- private static final Logger LOG = LoggerFactory.getLogger(BatchedReadOp.class);
-
final int maxCount;
final long maxSize;
@@ -80,11 +78,16 @@ protected void submitCallback(int code) {
long latencyNanos = MathUtils.elapsedNanos(requestTimeNanos);
if (code != BKException.Code.OK) {
- LOG.error(
- "Batch read of ledger entry failed: L{} E{}-E{}, Sent to {}, "
- + "Heard from {} : bitset = {}, Error = '{}'. First unread entry is ({}, rc = {})",
- lh.getId(), startEntryId, startEntryId + maxCount - 1, sentToHosts, heardFromHosts,
- heardFromHostsBitSet, BKException.getMessage(code), startEntryId, code);
+ log.error()
+ .attr("ledgerId", lh.getId())
+ .attr("startEntryId", startEntryId)
+ .attr("entryId", startEntryId + maxCount - 1)
+ .attr("sentToHosts", sentToHosts)
+ .attr("heardFromHosts", heardFromHosts)
+ .attr("heardFromHostsBitSet", heardFromHostsBitSet)
+ .attr("message", BKException.getMessage(code))
+ .attr("code", code)
+ .log("Batch read of ledger entry failed");
clientCtx.getClientStats().getReadOpLogger().registerFailedEvent(latencyNanos, TimeUnit.NANOSECONDS);
// release the entries
@@ -274,7 +277,10 @@ synchronized BookieId sendNextRead() {
sentReplicas.set(replica);
return to;
} catch (InterruptedException ie) {
- LOG.error("Interrupted reading entry " + this, ie);
+ log.error()
+ .attr("readOp", this)
+ .exception(ie)
+ .log("Interrupted reading entry");
Thread.currentThread().interrupt();
fail(BKException.Code.InterruptedException);
return null;
@@ -286,7 +292,10 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, Strin
super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc);
int replica = writeSet.indexOf(bookieIndex);
if (replica == NOT_FOUND) {
- LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble);
+ log.error()
+ .attr("bookieAddr", host)
+ .attr("ensemble", ensemble)
+ .log("Received error from a host which is not in the ensemble");
return;
}
erroredReplicas.set(replica);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index 9c70f582983..e34a2ebefa4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -44,6 +44,7 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookKeeperServerStats;
import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback;
@@ -92,8 +93,6 @@
import org.apache.commons.configuration2.ex.ConfigurationException;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* BookKeeper client.
@@ -106,10 +105,9 @@
*
The exceptions resulting from synchronous calls and error code resulting from
* asynchronous calls can be found in the class {@link BKException}.
*/
+@CustomLog
public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
- private static final Logger LOG = LoggerFactory.getLogger(BookKeeper.class);
-
final EventLoopGroup eventLoopGroup;
private final ByteBufAllocator allocator;
@@ -348,7 +346,7 @@ public BookKeeper(final ClientConfiguration conf)
private static ZooKeeper validateZooKeeper(ZooKeeper zk) throws NullPointerException, IOException {
checkNotNull(zk, "No zookeeper instance provided");
if (!zk.getState().isConnected()) {
- LOG.error("Unconnected zookeeper handle passed to bookkeeper");
+ log.error("Unconnected zookeeper handle passed to bookkeeper");
throw new IOException(KeeperException.create(KeeperException.Code.CONNECTIONLOSS));
}
return zk;
@@ -458,10 +456,12 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo
rootStatsLogger,
Optional.ofNullable(zkc));
} catch (ConfigurationException ce) {
- LOG.error("Failed to initialize metadata client driver using invalid metadata service uri", ce);
+ log.error()
+ .exception(ce)
+ .log("Failed to initialize metadata client driver using invalid metadata service uri");
throw new IOException("Failed to initialize metadata client driver", ce);
} catch (MetadataException me) {
- LOG.error("Encountered metadata exceptions on initializing metadata client driver", me);
+ log.error().exception(me).log("Encountered metadata exceptions on initializing metadata client driver");
throw new IOException("Failed to initialize metadata client driver", me);
}
@@ -518,7 +518,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo
scheduler, rootStatsLogger, this.bookieWatcher.getBookieAddressResolver());
if (conf.getDiskWeightBasedPlacementEnabled()) {
- LOG.info("Weighted ledger placement enabled");
+ log.info("Weighted ledger placement enabled");
ThreadFactoryBuilder tFBuilder = new ThreadFactoryBuilder()
.setNameFormat("BKClientMetaDataPollScheduler-%d");
this.bookieInfoScheduler = Executors.newSingleThreadScheduledExecutor(tFBuilder.build());
@@ -526,7 +526,7 @@ public BookKeeper(ClientConfiguration conf, ZooKeeper zk, EventLoopGroup eventLo
this.bookieWatcher.initialBlockingBookieRead();
this.bookieInfoReader.start();
} else {
- LOG.info("Weighted ledger placement is not enabled");
+ log.info("Weighted ledger placement is not enabled");
this.bookieInfoScheduler = null;
this.bookieInfoReader = new BookieInfoReader(this, conf, null);
this.bookieWatcher.initialBlockingBookieRead();
@@ -627,12 +627,12 @@ void checkForFaultyBookies() {
isEnabled = metadataDriver.isHealthCheckEnabled().get();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
- LOG.error("Cannot verify if healthcheck is enabled", e);
+ log.error().exception(e).log("Cannot verify if healthcheck is enabled");
} catch (ExecutionException e) {
- LOG.error("Cannot verify if healthcheck is enabled", e.getCause());
+ log.error().attr("getCause", e.getCause()).log("Cannot verify if healthcheck is enabled");
}
if (!isEnabled) {
- LOG.info("Health checks is currently disabled!");
+ log.info("Health checks is currently disabled!");
bookieWatcher.releaseAllQuarantinedBookies();
return;
}
@@ -967,7 +967,7 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum
LedgerHandle lh = SyncCallbackUtils.waitForResult(future);
if (lh == null) {
- LOG.error("Unexpected condition : no ledger handle returned for a success ledger creation");
+ log.error("Unexpected condition : no ledger handle returned for a success ledger creation");
throw BKException.create(BKException.Code.UnexpectedConditionException);
}
return lh;
@@ -1024,7 +1024,7 @@ public LedgerHandle createLedgerAdv(int ensSize, int writeQuorumSize, int ackQuo
LedgerHandle lh = SyncCallbackUtils.waitForResult(future);
if (lh == null) {
- LOG.error("Unexpected condition : no ledger handle returned for a success ledger creation");
+ log.error("Unexpected condition : no ledger handle returned for a success ledger creation");
throw BKException.create(BKException.Code.UnexpectedConditionException);
}
return lh;
@@ -1117,14 +1117,20 @@ public LedgerHandle createLedgerAdv(final long ledgerId,
LedgerHandle lh = SyncCallbackUtils.waitForResult(future);
if (lh == null) {
- LOG.error("Unexpected condition : no ledger handle returned for a success ledger creation");
+ log.error("Unexpected condition : no ledger handle returned for a success ledger creation");
throw BKException.create(BKException.Code.UnexpectedConditionException);
} else if (ledgerId != lh.getId()) {
- LOG.error("Unexpected condition : Expected ledgerId: {} but got: {}", ledgerId, lh.getId());
+ log.error()
+ .attr("expectedLedgerId", ledgerId)
+ .attr("gotLedgerId", lh.getId())
+ .log("Unexpected condition");
throw BKException.create(BKException.Code.UnexpectedConditionException);
}
- LOG.info("Ensemble: {} for ledger: {}", lh.getLedgerMetadata().getEnsembleAt(0L), lh.getId());
+ log.info()
+ .attr("getEnsembleAt", lh.getLedgerMetadata().getEnsembleAt(0L))
+ .attr("ledgerId", lh.getId())
+ .log("Ensemble for ledger");
return lh;
}
@@ -1521,30 +1527,30 @@ public void close() throws BKException, InterruptedException {
ledgerManager.close();
ledgerIdGenerator.close();
} catch (IOException ie) {
- LOG.error("Failed to close ledger manager : ", ie);
+ log.error().exception(ie).log("Failed to close ledger manager");
}
// Close the scheduler
scheduler.shutdown();
if (!scheduler.awaitTermination(10, TimeUnit.SECONDS)) {
- LOG.warn("The scheduler did not shutdown cleanly");
+ log.warn("The scheduler did not shutdown cleanly");
}
// Close the watchTask scheduler
highPriorityTaskExecutor.shutdown();
if (!highPriorityTaskExecutor.awaitTermination(10, TimeUnit.SECONDS)) {
- LOG.warn("The highPriorityTaskExecutor for WatchTask did not shutdown cleanly, interrupting");
+ log.warn("The highPriorityTaskExecutor for WatchTask did not shutdown cleanly, interrupting");
highPriorityTaskExecutor.shutdownNow();
}
mainWorkerPool.shutdown();
if (!mainWorkerPool.awaitTermination(10, TimeUnit.SECONDS)) {
- LOG.warn("The mainWorkerPool did not shutdown cleanly");
+ log.warn("The mainWorkerPool did not shutdown cleanly");
}
if (this.bookieInfoScheduler != null) {
this.bookieInfoScheduler.shutdown();
if (!bookieInfoScheduler.awaitTermination(10, TimeUnit.SECONDS)) {
- LOG.warn("The bookieInfoScheduler did not shutdown cleanly");
+ log.warn("The bookieInfoScheduler did not shutdown cleanly");
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
index 7bc99b2370f..c0337f9b083 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
@@ -56,6 +56,7 @@
import java.util.function.BiConsumer;
import java.util.function.Function;
import java.util.function.Predicate;
+import lombok.CustomLog;
import lombok.SneakyThrows;
import org.apache.bookkeeper.bookie.BookieException;
import org.apache.bookkeeper.bookie.BookieImpl;
@@ -96,16 +97,13 @@
import org.apache.commons.collections4.MapUtils;
import org.apache.zookeeper.AsyncCallback;
import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Admin client for BookKeeper clusters.
*/
+@CustomLog
public class BookKeeperAdmin implements AutoCloseable {
-
- private static final Logger LOG = LoggerFactory.getLogger(BookKeeperAdmin.class);
- private static final Logger VERBOSE = LoggerFactory.getLogger("verbose");
+ private static final io.github.merlimat.slog.Logger VERBOSE = io.github.merlimat.slog.Logger.get("verbose");
private static final BiConsumer NOOP_BICONSUMER = (l, e) -> { };
// BookKeeper client instance
@@ -457,7 +455,11 @@ public boolean hasNext() {
close();
return false;
}
- LOG.error("Error reading entry {} from ledger {}", nextEntryId, ledgerId, e);
+ log.error()
+ .exception(e)
+ .attr("entryId", nextEntryId)
+ .attr("ledgerId", ledgerId)
+ .log("Error reading entry from ledger");
close();
throw new RuntimeException(e);
}
@@ -487,7 +489,10 @@ private void close() {
try {
handle.close();
} catch (Exception e) {
- LOG.error("Error closing ledger handle {}", handle, e);
+ log.error()
+ .exception(e)
+ .attr("handle", handle)
+ .log("Error closing ledger handle");
}
}
}
@@ -512,7 +517,7 @@ public SortedMap getLedgersContainBookies(Set bo
asyncGetLedgersContainBookies(bookies, new GenericCallback>() {
@Override
public void operationComplete(int rc, SortedMap result) {
- LOG.info("GetLedgersContainBookies completed with rc : {}", rc);
+ log.info().attr("returnCode", rc).log("GetLedgersContainBookies completed");
synchronized (sync) {
sync.rc = rc;
sync.value = true;
@@ -605,7 +610,9 @@ public void recoverBookieData(final Set bookiesSrc, boolean dryrun, bo
asyncRecoverBookieData(bookiesSrc, dryrun, skipOpenLedgers, skipUnrecoverableLedgers, new RecoverCallback() {
@Override
public void recoverComplete(int rc, Object ctx) {
- LOG.info("Recover bookie operation completed with rc: {}", BKException.codeLogger(rc));
+ log.info()
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Recover bookie operation completed");
SyncObject syncObj = (SyncObject) ctx;
synchronized (syncObj) {
syncObj.rc = rc;
@@ -634,7 +641,10 @@ public void recoverBookieData(final long lid,
SyncObject sync = new SyncObject();
// Call the async method to recover bookie data.
asyncRecoverBookieData(lid, bookiesSrc, dryrun, skipOpenLedgers, (rc, ctx) -> {
- LOG.info("Recover bookie for {} completed with rc : {}", lid, BKException.codeLogger(rc));
+ log.info()
+ .attr("ledgerId", lid)
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Recover bookie completed");
SyncObject syncObject = (SyncObject) ctx;
synchronized (syncObject) {
syncObject.rc = rc;
@@ -807,19 +817,25 @@ private void recoverLedger(final Set bookiesSrc, final long lId, final
private void recoverLedger(final Set bookiesSrc, final long lId, final boolean dryrun,
final boolean skipOpenLedgers, final boolean skipUnrecoverableLedgers,
final AsyncCallback.VoidCallback finalLedgerIterCb) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Recovering ledger : {}", lId);
- }
+
+ log.debug().attr("ledgerId", lId).log("Recovering ledger");
+
asyncOpenLedgerNoRecovery(lId, new OpenCallback() {
@Override
public void openComplete(int rc, final LedgerHandle lh, Object ctx) {
if (rc != BKException.Code.OK) {
if (skipUnrecoverableLedgers) {
- LOG.warn("BK error opening ledger: {}, skip recover it.", lId, BKException.create(rc));
+ log.warn()
+ .attr("ledgerId", lId)
+ .attr("create", BKException.create(rc))
+ .log("BK error opening ledger, skip recover it");
finalLedgerIterCb.processResult(BKException.Code.OK, null, null);
} else {
- LOG.error("BK error opening ledger: {}", lId, BKException.create(rc));
+ log.error()
+ .attr("ledgerId", lId)
+ .attr("create", BKException.create(rc))
+ .log("BK error opening ledger");
finalLedgerIterCb.processResult(rc, null, null);
}
return;
@@ -827,13 +843,13 @@ public void openComplete(int rc, final LedgerHandle lh, Object ctx) {
LedgerMetadata lm = lh.getLedgerMetadata();
if (skipOpenLedgers && lm.getState() == LedgerMetadata.State.OPEN) {
- LOG.info("Skip recovering open ledger {}.", lId);
+ log.info().attr("ledgerId", lId).log("Skip recovering open ledger");
try {
lh.close();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
} catch (BKException bke) {
- LOG.warn("Error on closing ledger handle for {}.", lId);
+ log.warn().attr("ledgerId", lId).log("Error on closing ledger handle");
}
finalLedgerIterCb.processResult(BKException.Code.OK, null, null);
return;
@@ -849,18 +865,23 @@ public void openComplete(int rc, final LedgerHandle lh, Object ctx) {
try {
lh.close();
} catch (Exception ie) {
- LOG.warn("Error closing non recovery ledger handle for ledger " + lId, ie);
+ log.warn().exception(ie).log("Error closing non recovery ledger handle for ledger " + lId);
}
asyncOpenLedger(lId, new OpenCallback() {
@Override
public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) {
if (newrc != BKException.Code.OK) {
if (skipUnrecoverableLedgers) {
- LOG.warn("BK error opening ledger: {}, skip recover it.",
- lId, BKException.create(newrc));
+ log.warn()
+ .attr("ledgerId", lId)
+ .attr("create", BKException.create(newrc))
+ .log("BK error opening ledger, skip recover it");
finalLedgerIterCb.processResult(BKException.Code.OK, null, null);
} else {
- LOG.error("BK error close ledger: {}", lId, BKException.create(newrc));
+ log.error()
+ .attr("ledgerId", lId)
+ .attr("create", BKException.create(newrc))
+ .log("BK error close ledger");
finalLedgerIterCb.processResult(newrc, null, null);
}
return;
@@ -880,21 +901,26 @@ public void openComplete(int newrc, final LedgerHandle newlh, Object newctx) {
public void processResult(int rc, String path, Object ctx) {
if (BKException.Code.OK != rc) {
if (skipUnrecoverableLedgers) {
- LOG.warn("Failed to recover ledger: {} : {}, skip recover it.", lId,
- BKException.codeLogger(rc));
+ log.warn()
+ .attr("ledgerId", lId)
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Failed to recover ledger, skip recover it");
rc = BKException.Code.OK;
} else {
- LOG.error("Failed to recover ledger {} : {}", lId, BKException.codeLogger(rc));
+ log.error()
+ .attr("ledgerId", lId)
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Failed to recover ledger");
}
} else {
- LOG.info("Recovered ledger {}.", lId);
+ log.info().attr("ledgerId", lId).log("Recovered ledger");
}
try {
lh.close();
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
} catch (BKException bke) {
- LOG.warn("Error on closing ledger handle for {}.", lId);
+ log.warn().attr("ledgerId", lId).log("Error on closing ledger handle");
}
finalLedgerIterCb.processResult(rc, path, ctx);
}
@@ -947,7 +973,10 @@ public void processResult(int rc, String path, Object ctx) {
}
if (dryrun) {
- VERBOSE.info("Recovered ledger {} : {}", lId, (fenceRequired ? "[fence required]" : ""));
+ VERBOSE.info()
+ .attr("ledgerId", lId)
+ .attr("status", (fenceRequired ? "[fence required]" : ""))
+ .log("Recovered ledger");
}
/*
@@ -971,8 +1000,11 @@ public void processResult(int rc, String path, Object ctx) {
if (!dryrun) {
ledgerFragmentsMcb.processResult(BKException.Code.NotEnoughBookiesException, null, null);
} else {
- VERBOSE.info(" Fragment [{} - {}] : {}", startEntryId, endEntryId,
- BKException.getMessage(BKException.Code.NotEnoughBookiesException));
+ VERBOSE.info()
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .attr("error", BKException.getMessage(BKException.Code.NotEnoughBookiesException))
+ .log("Fragment recovery failed");
}
continue;
}
@@ -980,14 +1012,25 @@ public void processResult(int rc, String path, Object ctx) {
if (dryrun) {
ArrayList newEnsemble =
replaceBookiesInEnsemble(ensemble, targetBookieAddresses);
- VERBOSE.info(" Fragment [{} - {}] : ", startEntryId, endEntryId);
- VERBOSE.info(" old ensemble : {}", formatEnsemble(ensemble, bookiesSrc, '*'));
- VERBOSE.info(" new ensemble : {}", formatEnsemble(newEnsemble, bookiesSrc, '*'));
+ VERBOSE.info()
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .log("Fragment");
+ VERBOSE.info()
+ .attr("oldEnsemble", formatEnsemble(ensemble, bookiesSrc, '*'))
+ .log("Old ensemble");
+ VERBOSE.info()
+ .attr("newEnsemble", formatEnsemble(newEnsemble, bookiesSrc, '*'))
+ .log("New ensemble");
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Replicating fragment from [{}, {}] of ledger {} to {}",
- startEntryId, endEntryId, lh.getId(), targetBookieAddresses);
- }
+
+ log.debug()
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .attr("ledgerId", lh.getId())
+ .attr("bookieAddr", targetBookieAddresses)
+ .log("Replicating fragment");
+
try {
LedgerFragmentReplicator.SingleFragmentCallback cb =
new LedgerFragmentReplicator.SingleFragmentCallback(ledgerFragmentsMcb, lh,
@@ -1104,11 +1147,12 @@ private Map getReplacementBookiesByIndexes(
bookiesToExclude);
BookieId newBookie = replaceBookieResponse.getResult();
PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getAdheringToPolicy();
- if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL && LOG.isDebugEnabled()) {
- LOG.debug(
- "replaceBookie for bookie: {} in ensemble: {} "
- + "is not adhering to placement policy and chose {}",
- oldBookie, ensemble, newBookie);
+ if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
+ log.debug()
+ .attr("oldBookie", oldBookie)
+ .attr("ensemble", ensemble)
+ .attr("newBookie", newBookie)
+ .log("replaceBookie is not adhering to placement policy");
}
targetBookieAddresses.put(bookieIndex, newBookie);
bookiesToExclude.add(newBookie);
@@ -1148,8 +1192,10 @@ public void replicateLedgerFragment(LedgerHandle lh, final LedgerFragment ledger
ledgerFragment.getBookiesIndexes().addAll(targetBookieAddresses.keySet());
}
if (MapUtils.isEmpty(targetBookieAddresses)) {
- LOG.warn("Could not replicate for {} ledger: {}, not find target bookie.",
- ledgerFragment.getReplicateType(), ledgerFragment.getLedgerId());
+ log.warn()
+ .attr("getReplicateType", ledgerFragment.getReplicateType())
+ .attr("ledgerId", ledgerFragment.getLedgerId())
+ .log("Could not replicate for ledger, could not find target bookie");
throw new BKException.BKLedgerRecoveryException();
}
replicateLedgerFragment(lh, ledgerFragment, targetBookieAddresses, onReadEntryFailureCallback);
@@ -1331,8 +1377,10 @@ public static boolean nukeExistingCluster(ServerConfiguration conf, String ledge
boolean force) throws Exception {
String confLedgersRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
if (!confLedgersRootPath.equals(ledgersRootPath)) {
- LOG.error("Provided ledgerRootPath : {} is not matching with config's ledgerRootPath: {}, "
- + "so exiting nuke operation", ledgersRootPath, confLedgersRootPath);
+ log.error()
+ .attr("ledgersRootPath", ledgersRootPath)
+ .attr("confLedgersRootPath", confLedgersRootPath)
+ .log("Provided ledgerRootPath does not match config's ledgerRootPath, exiting nuke operation");
return false;
}
@@ -1341,8 +1389,10 @@ public static boolean nukeExistingCluster(ServerConfiguration conf, String ledge
if (!force) {
String readInstanceId = rm.getClusterInstanceId();
if ((instanceId == null) || !instanceId.equals(readInstanceId)) {
- LOG.error("Provided InstanceId : {} is not matching with cluster InstanceId in ZK: {}",
- instanceId, readInstanceId);
+ log.error()
+ .attr("instanceId", instanceId)
+ .attr("readInstanceId", readInstanceId)
+ .log("Provided InstanceId is not matching with cluster InstanceId in ZK");
return false;
}
}
@@ -1390,14 +1440,18 @@ public static boolean initBookie(ServerConfiguration conf) throws Exception {
*/
BookieId bookieId = BookieImpl.getBookieId(conf);
if (rm.isBookieRegistered(bookieId)) {
- LOG.error("Bookie with bookieId: {} is still registered, "
- + "If this node is running bookie process, try stopping it first.", bookieId);
+ log.error()
+ .attr("bookieId", bookieId)
+ .log("Bookie with bookieId: is still registered, "
+ + "If this node is running bookie process, try stopping it first.");
return false;
}
try {
rm.readCookie(bookieId);
- LOG.error("Cookie still exists in the ZK for this bookie: {}, try formatting the bookie", bookieId);
+ log.error()
+ .attr("bookieId", bookieId)
+ .log("Cookie still exists in the ZK for this bookie, try formatting the bookie");
return false;
} catch (BookieException.CookieNotFoundException nfe) {
// it is expected for readCookie to fail with
@@ -1414,7 +1468,10 @@ private static boolean validateDirectoriesAreEmpty(File[] dirs, String typeOfDir
for (File dir : dirs) {
File[] dirFiles = dir.listFiles();
if ((dirFiles != null) && dirFiles.length != 0) {
- LOG.error("{}: {} is existing and its not empty, try formatting the bookie", typeOfDir, dir);
+ log.error()
+ .attr("typeOfDir", typeOfDir)
+ .attr("directory", dir)
+ .log(": is existing and its not empty, try formatting the bookie");
return false;
}
}
@@ -1449,7 +1506,7 @@ public boolean hasNext() {
}
}
} catch (IOException e) {
- LOG.error("Error while checking if there is a next element", e);
+ log.error().exception(e).log("Error while checking if there is a next element");
}
return false;
@@ -1462,7 +1519,7 @@ public Long next() throws NoSuchElementException {
currentRange = iterator.next().getLedgers().iterator();
}
} catch (IOException e) {
- LOG.error("Error while reading the next element", e);
+ log.error().exception(e).log("Error while reading the next element");
throw new NoSuchElementException(e.getMessage());
}
@@ -1549,7 +1606,7 @@ public void triggerAudit()
throws CompatibilityException, KeeperException, InterruptedException, UnavailableException, IOException {
LedgerUnderreplicationManager urlManager = getUnderreplicationManager();
if (!urlManager.isLedgerReplicationEnabled()) {
- LOG.error("Autorecovery is disabled. So giving up!");
+ log.error("Autorecovery is disabled. So giving up!");
throw new UnavailableException("Autorecovery is disabled. So giving up!");
}
@@ -1558,20 +1615,21 @@ public void triggerAudit()
auditorId = getLedgerAuditorManager().getCurrentAuditor();
} catch (IOException e) {
if (e.getCause() instanceof KeeperException.NoNodeException) {
- LOG.error("Unable to find Zookeeper node: {}", e.getCause().getMessage());
+ log.error().attr("getMessage", e.getCause().getMessage()).log("Unable to find Zookeeper node");
throw new UnavailableException("Autorecovery is disabled due to "
+ "missing Zookeeper node. Aborting recovery!");
}
throw e;
}
if (auditorId == null) {
- LOG.error("No auditor elected, though Autorecovery is enabled. So giving up.");
+ log.error("No auditor elected, though Autorecovery is enabled. So giving up.");
throw new UnavailableException("No auditor elected, though Autorecovery is enabled. So giving up.");
}
int previousLostBookieRecoveryDelayValue = urlManager.getLostBookieRecoveryDelay();
- LOG.info("Resetting LostBookieRecoveryDelay value: {}, to kickstart audit task",
- previousLostBookieRecoveryDelayValue);
+ log.info()
+ .attr("previousLostBookieRecoveryDelayValue", previousLostBookieRecoveryDelayValue)
+ .log("Resetting LostBookieRecoveryDelay to kickstart audit task");
urlManager.setLostBookieRecoveryDelay(previousLostBookieRecoveryDelayValue);
}
@@ -1598,7 +1656,7 @@ public void decommissionBookie(BookieId bookieAddress)
throws CompatibilityException, UnavailableException, KeeperException, InterruptedException, IOException,
BKAuditException, TimeoutException, BKException {
if (getAvailableBookies().contains(bookieAddress) || getReadOnlyBookies().contains(bookieAddress)) {
- LOG.error("Bookie: {} is not shutdown yet", bookieAddress);
+ log.error().attr("bookieAddress", bookieAddress).log("Bookie: is not shutdown yet");
throw BKException.create(BKException.Code.IllegalOpException);
}
@@ -1634,7 +1692,7 @@ public void decommissionBookie(BookieId bookieAddress)
Iterator urLedgerIterator = underreplicationManager.listLedgersToRereplicate(predicate);
if (urLedgerIterator.hasNext()) {
//if there are any then wait and make sure those ledgers are replicated properly
- LOG.info("Still in some underreplicated ledgers metadata, this bookie is part of its ensemble. "
+ log.info("Still in some underreplicated ledgers metadata, this bookie is part of its ensemble. "
+ "Have to make sure that those ledger fragments are rereplicated");
List urLedgers = new ArrayList<>();
urLedgerIterator.forEachRemaining((urLedger) -> {
@@ -1655,12 +1713,14 @@ private void waitForLedgersToBeReplicated(Collection ledgers, BookieId thi
while (!ledgers.isEmpty()) {
int sleepTimeForThisCheck = (long) ledgers.size() * sleepTimePerLedger > maxSleepTimeInBetweenChecks
? maxSleepTimeInBetweenChecks : ledgers.size() * sleepTimePerLedger;
- LOG.info("Count of Ledgers which need to be rereplicated: {}, waiting {} seconds for next check",
- ledgers.size(), sleepTimeForThisCheck / 1000);
+ log.info()
+ .attr("size", ledgers.size())
+ .attr("value", sleepTimeForThisCheck / 1000)
+ .log("Count of ledgers which need to be rereplicated, waiting for next check");
Thread.sleep(sleepTimeForThisCheck);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Making sure following ledgers replication to be completed: {}", ledgers);
- }
+
+ log.debug().attr("ledgers", ledgers).log("Making sure ledger replication completes");
+
ledgers.removeIf(validateBookieIsNotPartOfEnsemble);
}
}
@@ -1677,12 +1737,13 @@ public static boolean areEntriesOfLedgerStoredInTheBookie(long ledgerId, BookieI
if (e.getCause() != null
&& e.getCause().getClass()
.equals(BKException.BKNoSuchLedgerExistsOnMetadataServerException.class)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Ledger: {} has been deleted", ledgerId);
- }
+
+ log.debug().attr("ledgerId", ledgerId).log("Ledger: has been deleted");
+
return false;
} else {
- LOG.error("Got exception while trying to read LedgerMetadata of " + ledgerId, e);
+ log.error().exception(e).attr("ledgerId", ledgerId)
+ .log("Got exception while trying to read LedgerMetadata");
throw new RuntimeException(e);
}
}
@@ -1828,8 +1889,10 @@ public Map replaceNotAdheringPlacementPolicyBookie(ListNOTE: This class is tended to be used by this project only. External users should not rely on it directly.
*/
+@CustomLog
public class BookieInfoReader {
- private static final Logger LOG = LoggerFactory.getLogger(BookieInfoReader.class);
private static final long GET_BOOKIE_INFO_REQUEST_FLAGS =
BookkeeperProtocol.GetBookieInfoRequest.Flags.TOTAL_DISK_CAPACITY_VALUE
| BookkeeperProtocol.GetBookieInfoRequest.Flags.FREE_DISK_SPACE_VALUE;
@@ -105,11 +104,12 @@ private static class BookieInfoMap {
private Collection mostRecentlyReportedBookies = new ArrayList<>();
public void updateBookies(Collection updatedBookieSet) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "updateBookies: current: {}, new: {}",
- mostRecentlyReportedBookies, updatedBookieSet);
- }
+
+ log.debug()
+ .attr("mostRecentlyReportedBookies", mostRecentlyReportedBookies)
+ .attr("updatedBookieSet", updatedBookieSet)
+ .log("updateBookies");
+
infoMap.keySet().retainAll(updatedBookieSet);
mostRecentlyReportedBookies = updatedBookieSet;
}
@@ -242,15 +242,18 @@ public void start() {
@Override
public void run() {
synchronized (BookieInfoReader.this) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Running periodic BookieInfo scan");
- }
+
+ log.debug("Running periodic BookieInfo scan");
+
try {
Collection updatedBookies = bk.bookieWatcher.getBookies();
bookieInfoMap.updateBookies(updatedBookies);
} catch (BKException e) {
- LOG.info("Got exception while querying bookies from watcher, rerunning after {}s",
- conf.getGetBookieInfoRetryIntervalSeconds(), e);
+ log.info()
+ .exception(e)
+ .attr("getGetBookieInfoRetryIntervalSeconds",
+ conf.getGetBookieInfoRetryIntervalSeconds())
+ .log("Got exception while querying bookies from watcher");
scheduler.schedule(this, conf.getGetBookieInfoRetryIntervalSeconds(), TimeUnit.SECONDS);
return;
}
@@ -271,9 +274,9 @@ private void submitTaskWithDelay(int delaySeconds) {
}
synchronized void availableBookiesChanged(Set updatedBookiesList) {
- if (LOG.isInfoEnabled()) {
- LOG.info("Scheduling bookie info read due to changes in available bookies.");
- }
+
+ log.info("Scheduling bookie info read due to changes in available bookies.");
+
bookieInfoMap.updateBookies(updatedBookiesList);
if (instanceState.tryStartPartial()) {
submitTask();
@@ -308,19 +311,19 @@ synchronized void getReadWriteBookieInfo() {
State queuedType = instanceState.getAndClearQueuedType();
Collection toScan;
if (queuedType == State.FULL) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Doing full scan");
- }
+
+ log.debug("Doing full scan");
+
toScan = bookieInfoMap.getFullScanTargets();
} else if (queuedType == State.PARTIAL) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Doing partial scan");
- }
+
+ log.debug("Doing partial scan");
+
toScan = bookieInfoMap.getPartialScanTargets();
} else {
- if (LOG.isErrorEnabled()) {
- LOG.error("Invalid state, queuedType cannot be UNQUEUED in getReadWriteBookieInfo");
- }
+
+ log.error("Invalid state, queuedType cannot be UNQUEUED in getReadWriteBookieInfo");
+
assert(queuedType != State.UNQUEUED);
return;
}
@@ -332,9 +335,9 @@ synchronized void getReadWriteBookieInfo() {
completedCnt = 0;
errorCnt = 0;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Getting bookie info for: {}", toScan);
- }
+
+ log.debug().attr("toScan", toScan).log("Getting bookie info");
+
for (BookieId b : toScan) {
bkc.getBookieInfo(b, requested,
new GetBookieInfoCallback() {
@@ -342,18 +345,23 @@ void processReadInfoComplete(int rc, BookieInfo bInfo, Object ctx) {
synchronized (BookieInfoReader.this) {
BookieId b = (BookieId) ctx;
if (rc != BKException.Code.OK) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Reading bookie info from bookie {} failed due to {}",
- b, BKException.codeLogger(rc));
- }
+
+ log.error()
+ .attr("bookieId", b)
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Reading bookie info from bookie failed");
+
// We reread bookies missing from the map each time, so remove to ensure
// we get to it on the next scan
bookieInfoMap.clearInfo(b);
errorCnt++;
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Bookie Info for bookie {} is {}", b, bInfo);
- }
+
+ log.debug()
+ .attr("bookieId", b)
+ .attr("bInfo", bInfo)
+ .log("Bookie Info received");
+
bookieInfoMap.gotInfo(b, bInfo);
}
completedCnt++;
@@ -383,15 +391,17 @@ public void run() {
void onExit() {
bk.placementPolicy.updateBookieInfo(bookieInfoMap.getBookieMap());
if (errorCnt > 0) {
- if (LOG.isInfoEnabled()) {
- LOG.info("Rescheduling in {}s due to errors", conf.getGetBookieInfoIntervalSeconds());
- }
+
+ log.info()
+ .attr("getGetBookieInfoIntervalSeconds", conf.getGetBookieInfoIntervalSeconds())
+ .log("Rescheduling due to errors");
+
instanceState.tryStartPartial();
submitTaskWithDelay(conf.getGetBookieInfoRetryIntervalSeconds());
} else if (instanceState.completeUnlessQueued()) {
- if (LOG.isInfoEnabled()) {
- LOG.info("Rescheduling, another scan is pending");
- }
+
+ log.info("Rescheduling, another scan is pending");
+
submitTask();
}
}
@@ -419,14 +429,19 @@ Map getBookieInfo() throws BKException, InterruptedExcepti
public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) {
BookieId b = (BookieId) ctx;
if (rc != BKException.Code.OK) {
- if (LOG.isErrorEnabled()) {
- LOG.error("Reading bookie info from bookie {} failed due to {}",
- b, BKException.codeLogger(rc));
- }
+
+ log.error()
+ .attr("bookieId", b)
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .log("Reading bookie info from bookie failed");
+
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Free disk space on bookie {} is {}.", b, bInfo.getFreeDiskSpace());
- }
+
+ log.debug()
+ .attr("bookieId", b)
+ .attr("getFreeDiskSpace", bInfo.getFreeDiskSpace())
+ .log("Free disk space on bookie");
+
map.put(b, bInfo);
}
if (totalCompleted.incrementAndGet() == totalSent.get()) {
@@ -439,7 +454,7 @@ public void getBookieInfoComplete(int rc, BookieInfo bInfo, Object ctx) {
latch.await();
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
- LOG.error("Received InterruptedException ", e);
+ log.error().exception(e).log("Received InterruptedException ");
throw e;
}
return map;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java
index f7af1549286..3b4011d8bdd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookieWatcherImpl.java
@@ -35,7 +35,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
-import lombok.extern.slf4j.Slf4j;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookKeeperServerStats;
import org.apache.bookkeeper.client.BKException.BKInterruptedException;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
@@ -63,15 +63,15 @@
name = WATCHER_SCOPE,
help = "Bookie watcher related stats"
)
-@Slf4j
+@CustomLog
class BookieWatcherImpl implements BookieWatcher {
private static final Function EXCEPTION_FUNC = cause -> {
if (cause instanceof BKException) {
- log.error("Failed to get bookie list : ", cause);
+ log.error().exception(cause).log("Failed to get bookie list");
return (BKException) cause;
} else if (cause instanceof InterruptedException) {
- log.error("Interrupted reading bookie list : ", cause);
+ log.error().exception(cause).log("Interrupted reading bookie list");
return new BKInterruptedException();
} else {
MetaStoreException mse = new MetaStoreException(cause);
@@ -126,7 +126,7 @@ public BookieWatcherImpl(ClientConfiguration conf,
@Override
public void onRemoval(RemovalNotification bookie) {
- log.info("Bookie {} is no longer quarantined", bookie.getKey());
+ log.info().attr("bookieId", bookie.getKey()).log("Bookie is no longer quarantined");
}
}).build();
@@ -252,7 +252,7 @@ public void initialBlockingBookieRead() throws BKException {
Thread.currentThread().interrupt();
throw ie;
} catch (Exception e) {
- log.error("Failed getReadOnlyBookies: ", e);
+ log.error().exception(e).log("Failed getReadOnlyBookies");
}
}
@@ -273,23 +273,27 @@ public List newEnsemble(int ensembleSize, int writeQuorumSize,
if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
ensembleNotAdheringToPlacementPolicy.inc();
if (ensembleSize > 1) {
- log.info("New ensemble: {} is not adhering to Placement Policy. quarantinedBookies: {}",
- socketAddresses, quarantinedBookiesSet);
+ log.info()
+ .attr("socketAddresses", socketAddresses)
+ .attr("quarantinedBookies", quarantinedBookiesSet)
+ .log("New ensemble: is not adhering to Placement Policy");
}
}
// we try to only get from the healthy bookies first
newEnsembleTimer.registerSuccessfulEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS);
} catch (BKNotEnoughBookiesException e) {
- if (log.isDebugEnabled()) {
- log.debug("Not enough healthy bookies available, using quarantined bookies");
- }
+
+ log.debug("Not enough healthy bookies available, using quarantined bookies");
+
newEnsembleResponse = placementPolicy.newEnsemble(
ensembleSize, writeQuorumSize, ackQuorumSize, customMetadata, new HashSet<>());
socketAddresses = newEnsembleResponse.getResult();
isEnsembleAdheringToPlacementPolicy = newEnsembleResponse.getAdheringToPolicy();
if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
ensembleNotAdheringToPlacementPolicy.inc();
- log.info("New ensemble: {} is not adhering to Placement Policy", socketAddresses);
+ log.info()
+ .attr("socketAddresses", socketAddresses)
+ .log("New ensemble: is not adhering to Placement Policy");
}
newEnsembleTimer.registerFailedEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS);
}
@@ -320,26 +324,31 @@ public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuor
isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getAdheringToPolicy();
if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
ensembleNotAdheringToPlacementPolicy.inc();
- log.warn(
- "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and"
- + " chose {}. excludedBookies {} and quarantinedBookies {}",
- addr, existingBookies, socketAddress, excludeBookies, quarantinedBookiesSet);
+ log.warn()
+ .attr("bookieAddr", addr)
+ .attr("existingBookies", existingBookies)
+ .attr("socketAddress", socketAddress)
+ .attr("excludeBookies", excludeBookies)
+ .attr("quarantinedBookiesSet", quarantinedBookiesSet)
+ .log("replaceBookie for bookie: in ensemble: is not adhering to placement policy and chose");
}
replaceBookieTimer.registerSuccessfulEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS);
} catch (BKNotEnoughBookiesException e) {
- if (log.isDebugEnabled()) {
- log.debug("Not enough healthy bookies available, using quarantined bookies");
- }
+
+ log.debug("Not enough healthy bookies available, using quarantined bookies");
+
replaceBookieResponse = placementPolicy.replaceBookie(ensembleSize, writeQuorumSize, ackQuorumSize,
customMetadata, existingBookies, addr, excludeBookies);
socketAddress = replaceBookieResponse.getResult();
isEnsembleAdheringToPlacementPolicy = replaceBookieResponse.getAdheringToPolicy();
if (isEnsembleAdheringToPlacementPolicy == PlacementPolicyAdherence.FAIL) {
ensembleNotAdheringToPlacementPolicy.inc();
- log.warn(
- "replaceBookie for bookie: {} in ensemble: {} is not adhering to placement policy and"
- + " chose {}. excludedBookies {}",
- addr, existingBookies, socketAddress, excludeBookies);
+ log.warn()
+ .attr("bookieAddr", addr)
+ .attr("existingBookies", existingBookies)
+ .attr("socketAddress", socketAddress)
+ .attr("excludeBookies", excludeBookies)
+ .log("replaceBookie for bookie: in ensemble: is not adhering to placement policy and chose");
}
replaceBookieTimer.registerFailedEvent(MathUtils.nowInNano() - startTime, TimeUnit.NANOSECONDS);
}
@@ -354,7 +363,7 @@ public BookieId replaceBookie(int ensembleSize, int writeQuorumSize, int ackQuor
public void quarantineBookie(BookieId bookie) {
if (quarantinedBookies.getIfPresent(bookie) == null) {
quarantinedBookies.put(bookie, Boolean.TRUE);
- log.warn("Bookie {} has been quarantined because of read/write errors.", bookie);
+ log.warn().attr("bookieId", bookie).log("Bookie has been quarantined because of read/write errors.");
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java
index 08016bdd821..7887cf2f723 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultBookieAddressResolver.java
@@ -17,7 +17,7 @@
*/
package org.apache.bookkeeper.client;
-import lombok.extern.slf4j.Slf4j;
+import lombok.CustomLog;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
import org.apache.bookkeeper.discover.BookieServiceInfo;
import org.apache.bookkeeper.discover.RegistrationClient;
@@ -28,7 +28,7 @@
/**
* Resolve BookieIDs to Network addresses.
*/
-@Slf4j
+@CustomLog
public class DefaultBookieAddressResolver implements BookieAddressResolver {
private final RegistrationClient registrationClient;
@@ -53,19 +53,24 @@ public BookieSocketAddress resolve(BookieId bookieId) {
BookieSocketAddress res = new BookieSocketAddress(endpoint.getHost(), endpoint.getPort());
if (!bookieId.toString().equals(res.toString())) {
// only print if the information is useful
- log.info("Resolved {} as {}", bookieId, res);
- } else if (log.isDebugEnabled()) {
- log.debug("Resolved {} as {}", bookieId, res);
+ log.info()
+ .attr("bookieId", bookieId)
+ .attr("res", res)
+ .log("Resolved as");
}
+
return res;
} catch (BKException.BKBookieHandleNotAvailableException ex) {
if (BookieSocketAddress.isDummyBookieIdForHostname(bookieId)) {
- if (log.isDebugEnabled()) {
- log.debug("Resolving dummy bookie Id {} using legacy bookie resolver", bookieId);
- }
+
+ log.debug().attr("bookieId", bookieId).log("Resolving dummy bookie Id using legacy bookie resolver");
+
return BookieSocketAddress.resolveLegacyBookieId(bookieId);
}
- log.info("Cannot resolve {}, bookie is unknown {}", bookieId, ex.toString());
+ log.info()
+ .attr("bookieId", bookieId)
+ .exceptionMessage(ex)
+ .log("Cannot resolve bookie, bookie is unknown");
throw new BookieIdNotResolvedException(bookieId, ex);
} catch (Exception ex) {
if (ex instanceof InterruptedException) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java
index 94cd30344f1..f54094cc7dd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultEnsemblePlacementPolicy.java
@@ -28,6 +28,7 @@
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject;
@@ -38,16 +39,14 @@
import org.apache.bookkeeper.proto.BookieAddressResolver;
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.commons.collections4.CollectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Default Ensemble Placement Policy, which picks bookies randomly.
*
* @see EnsemblePlacementPolicy
*/
+@CustomLog
public class DefaultEnsemblePlacementPolicy implements EnsemblePlacementPolicy {
- static final Logger LOG = LoggerFactory.getLogger(DefaultEnsemblePlacementPolicy.class);
static final Set EMPTY_SET = new HashSet();
private boolean isWeighted;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java
index c85fb1dc29e..cc2fe8a3376 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DefaultSpeculativeRequestExecutionPolicy.java
@@ -29,8 +29,7 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import lombok.CustomLog;
/**
* A default implementation of {@link SpeculativeRequestExecutionPolicy}.
@@ -38,8 +37,8 @@
* The policy issues speculative requests in a backoff way. The time between two speculative requests
* are between {@code firstSpeculativeRequestTimeout} and {@code maxSpeculativeRequestTimeout}.
*/
+@CustomLog
public class DefaultSpeculativeRequestExecutionPolicy implements SpeculativeRequestExecutionPolicy {
- private static final Logger LOG = LoggerFactory.getLogger(DefaultSpeculativeRequestExecutionPolicy.class);
final int firstSpeculativeRequestTimeout;
final int maxSpeculativeRequestTimeout;
final float backoffMultiplier;
@@ -90,25 +89,33 @@ public void onSuccess(Boolean issueNextRequest) {
Math.min(maxSpeculativeRequestTimeout,
Math.round((float) speculativeRequestTimeout * backoffMultiplier)));
} else {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Stopped issuing speculative requests for {}, "
- + "speculativeReadTimeout = {}", requestExecutor, speculativeRequestTimeout);
- }
+
+ log.trace()
+ .attr("requestExecutor", requestExecutor)
+ .attr("speculativeRequestTimeout", speculativeRequestTimeout)
+ .log("Stopped issuing speculative requests");
+
}
}
@Override
public void onFailure(Throwable thrown) {
- LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ",
- requestExecutor, speculativeRequestTimeout, thrown);
+ log.warn()
+ .attr("requestExecutor", requestExecutor)
+ .attr("speculativeRequestTimeout", speculativeRequestTimeout)
+ .exception(thrown)
+ .log("Failed to issue speculative request");
}
}, directExecutor());
}
}, speculativeRequestTimeout, TimeUnit.MILLISECONDS);
} catch (RejectedExecutionException re) {
if (!scheduler.isShutdown()) {
- LOG.warn("Failed to schedule speculative request for {}, speculativeReadTimeout = {} : ",
- requestExecutor, speculativeRequestTimeout, re);
+ log.warn()
+ .exception(re)
+ .attr("requestExecutor", requestExecutor)
+ .attr("speculativeRequestTimeout", speculativeRequestTimeout)
+ .log("Failed to schedule speculative request");
}
}
return null;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java
index 7da6b4c20d6..dc037cb3504 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/DynamicWeightedRandomSelectionImpl.java
@@ -28,8 +28,7 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Function;
import java.util.stream.Collectors;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import lombok.CustomLog;
/**
* DynamicWeightedRandomSelectionImpl class implements both getNextRandom
@@ -37,8 +36,8 @@
* knows of as candidates, but getNextRandom(Collection selectedNodes) method
* considers only 'selectedNodes' as candidates.
*/
+@CustomLog
class DynamicWeightedRandomSelectionImpl implements WeightedRandomSelection {
- static final Logger LOG = LoggerFactory.getLogger(DynamicWeightedRandomSelectionImpl.class);
int maxProbabilityMultiplier;
final Map weightMap;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
index dd16f800c61..79546840ae8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/EnsembleUtils.java
@@ -27,19 +27,18 @@
import java.util.List;
import java.util.Map;
import java.util.Set;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.net.BookieId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+@CustomLog
class EnsembleUtils {
- private static final Logger LOG = LoggerFactory.getLogger(EnsembleUtils.class);
static List replaceBookiesInEnsemble(BookieWatcher bookieWatcher,
LedgerMetadata metadata,
List oldEnsemble,
Map failedBookies,
- String logContext)
+ int ensembleChangeId)
throws BKException.BKNotEnoughBookiesException {
List newEnsemble = new ArrayList<>(oldEnsemble);
@@ -54,15 +53,23 @@ static List replaceBookiesInEnsemble(BookieWatcher bookieWatcher,
for (Map.Entry entry : failedBookies.entrySet()) {
int idx = entry.getKey();
BookieId addr = entry.getValue();
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} replacing bookie: {} index: {}", logContext, addr, idx);
- }
+
+ log.debug()
+ .attr("ledgerId", metadata.getLedgerId())
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("bookieAddr", addr)
+ .attr("idx", idx)
+ .log("replacing bookie");
if (!newEnsemble.get(idx).equals(addr)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} Not changing failed bookie {} at index {}, already changed to {}",
- logContext, addr, idx, newEnsemble.get(idx));
- }
+ log.debug()
+ .attr("ledgerId", metadata.getLedgerId())
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("bookieAddr", addr)
+ .attr("idx", idx)
+ .attr("newBookie", newEnsemble.get(idx))
+ .log("Not changing failed bookie at index, already changed");
+
continue;
}
try {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java
index e2c222220d2..d62d49a121b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ExplicitLacFlushPolicy.java
@@ -23,10 +23,9 @@
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.SyncCallbackUtils.LastAddConfirmedCallback;
import org.apache.bookkeeper.util.ByteBufList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
interface ExplicitLacFlushPolicy {
void stopExplicitLacFlush();
@@ -45,8 +44,8 @@ public void updatePiggyBackedLac(long piggyBackedLac) {
}
};
+ @CustomLog
class ExplicitLacFlushPolicyImpl implements ExplicitLacFlushPolicy {
- static final Logger LOG = LoggerFactory.getLogger(ExplicitLacFlushPolicyImpl.class);
volatile long piggyBackedLac = LedgerHandle.INVALID_ENTRY_ID;
volatile long explicitLac = LedgerHandle.INVALID_ENTRY_ID;
@@ -61,9 +60,9 @@ class ExplicitLacFlushPolicyImpl implements ExplicitLacFlushPolicy {
this.clientCtx = clientCtx;
scheduleExplictLacFlush();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Scheduled Explicit Last Add Confirmed Update");
- }
+
+ log.debug("Scheduled Explicit Last Add Confirmed Update");
+
}
private long getExplicitLac() {
@@ -90,25 +89,28 @@ public void run() {
// Piggyback, so no need to send an explicit LAC update to
// bookies.
if (getExplicitLac() < getPiggyBackedLac()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("ledgerid: {}", lh.getId());
- LOG.debug("explicitLac:{} piggybackLac:{}", getExplicitLac(), getPiggyBackedLac());
- }
+ log.debug()
+ .attr("ledgerId", lh.getId())
+ .attr("explicitLac", getExplicitLac())
+ .attr("piggyBackedLac", getPiggyBackedLac())
+ .log("explicitLac / piggybackLac");
+
setExplicitLac(getPiggyBackedLac());
return;
}
if (lh.getLastAddConfirmed() > getExplicitLac()) {
// Send Explicit LAC
- if (LOG.isDebugEnabled()) {
- LOG.debug("ledgerid: {}", lh.getId());
- }
+ log.debug().attr("ledgerId", lh.getId()).log("Send explicit LAC");
+
asyncExplicitLacFlush(lh.getLastAddConfirmed());
setExplicitLac(lh.getLastAddConfirmed());
- if (LOG.isDebugEnabled()) {
- LOG.debug("After sending explict LAC lac: {} explicitLac:{}", lh.getLastAddConfirmed(),
- getExplicitLac());
- }
+
+ log.debug()
+ .attr("ledgerId", lh.getId())
+ .attr("getLastAddConfirmed", lh.getLastAddConfirmed())
+ .attr("explicitLac", getExplicitLac())
+ .log("After sending explict LAC lac");
}
}
@@ -122,7 +124,10 @@ public String toString() {
scheduledFuture = clientCtx.getScheduler().scheduleAtFixedRateOrdered(lh.getId(), updateLacTask,
explicitLacIntervalInMs, explicitLacIntervalInMs, TimeUnit.MILLISECONDS);
} catch (RejectedExecutionException re) {
- LOG.error("Scheduling of ExplictLastAddConfirmedFlush for ledger: {} has failed.", lh.getId(), re);
+ log.error()
+ .exception(re)
+ .attr("ledgerId", lh.getId())
+ .log("Scheduling of ExplictLastAddConfirmedFlush for ledger: has failed.");
}
}
@@ -134,9 +139,9 @@ void asyncExplicitLacFlush(final long explicitLac) {
final PendingWriteLacOp op = new PendingWriteLacOp(lh, clientCtx, lh.getCurrentEnsemble(), cb, null);
op.setLac(explicitLac);
try {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Sending Explicit LAC: {}", explicitLac);
- }
+
+ log.debug().attr("explicitLac", explicitLac).log("Sending Explicit LAC");
+
clientCtx.getMainWorkerPool().submit(() -> {
ByteBufList toSend = lh.macManager
.computeDigestAndPackageForSendingLac(lh.getLastAddConfirmed());
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java
index b96fa095ddb..2597849d631 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ForceLedgerOp.java
@@ -21,19 +21,17 @@
import java.util.List;
import java.util.concurrent.CompletableFuture;
+import lombok.CustomLog;
import org.apache.bookkeeper.common.concurrent.FutureUtils;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ForceLedgerCallback;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This represents a request to sync the ledger on every bookie.
*/
+@CustomLog
class ForceLedgerOp implements Runnable, ForceLedgerCallback {
-
- private static final Logger LOG = LoggerFactory.getLogger(ForceLedgerOp.class);
final CompletableFuture cb;
DistributionSchedule.AckSet ackSet;
@@ -71,9 +69,12 @@ void initiate() {
// remember that we are inside OrderedExecutor, this induces a strict ordering
// on the sequence of events
this.currentNonDurableLastAddConfirmed = lh.pendingAddsSequenceHead;
- if (LOG.isDebugEnabled()) {
- LOG.debug("force {} clientNonDurableLac {}", lh.ledgerId, currentNonDurableLastAddConfirmed);
- }
+
+ log.debug()
+ .attr("ledgerId", lh.ledgerId)
+ .attr("nonDurableLac", currentNonDurableLastAddConfirmed)
+ .log("force clientNonDurableLac");
+
// we need to send the request to every bookie in the ensamble
this.ackSet = lh.distributionSchedule.getEnsembleAckSet();
@@ -108,17 +109,22 @@ public void forceLedgerComplete(int rc, long ledgerId, BookieId addr, Object ctx
completed = true;
// we are able to say that every bookie sync'd its own journal
// for every acknowledged entry before issuing the force() call
- if (LOG.isDebugEnabled()) {
- LOG.debug("After force on ledger {} updating LastAddConfirmed to {} ",
- ledgerId, currentNonDurableLastAddConfirmed);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("nonDurableLac", currentNonDurableLastAddConfirmed)
+ .log("After force on ledger, updating LastAddConfirmed");
+
lh.updateLastConfirmed(currentNonDurableLastAddConfirmed, lh.getLength());
FutureUtils.complete(cb, null);
}
} else {
// at least one bookie failed, as we are waiting for all the bookies
// we can fail immediately
- LOG.error("ForceLedger did not succeed: Ledger {} on {}", ledgerId, addr);
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("bookieAddr", addr)
+ .log("ForceLedger did not succeed");
errored = true;
// notify the failure
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java
index 9364b8563c6..ef12014a97f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java
@@ -33,22 +33,21 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.Code;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
import org.apache.bookkeeper.proto.BookieProtocol;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* A utility class to check the complete ledger and finds the UnderReplicated fragments if any.
*
* NOTE: This class is tended to be used by this project only. External users should not rely on it directly.
*/
+@CustomLog
public class LedgerChecker {
- private static final Logger LOG = LoggerFactory.getLogger(LedgerChecker.class);
public final BookieClient bookieClient;
public final BookieWatcher bookieWatcher;
@@ -87,9 +86,14 @@ public void readEntryComplete(int rc, long ledgerId, long entryId,
cb.operationComplete(rc, fragment);
}
} else if (!completed.getAndSet(true)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Read {}:{} from {} failed, the error code: {}", ledgerId, entryId, ctx, rc);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("ctx", ctx)
+ .attr("returnCode", rc)
+ .log("Read failed");
+
cb.operationComplete(rc, fragment);
}
}
@@ -455,7 +459,10 @@ public void operationComplete(int rc, Boolean result) {
bookieClient.readEntry(addr, lh.getId(), entryToRead,
eecb, null, BookieProtocol.FLAG_NONE);
} catch (InterruptedException e) {
- LOG.error("InterruptedException when checking entry : {}", entryToRead, e);
+ log.error()
+ .exception(e)
+ .attr("entryId", entryToRead)
+ .log("InterruptedException when checking entry");
}
}
return;
@@ -478,19 +485,25 @@ private void checkFragments(Set fragments,
FullLedgerCallback allFragmentsCb = new FullLedgerCallback(fragments
.size(), cb);
for (LedgerFragment r : fragments) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Checking fragment {}", r);
- }
+
+ log.debug().attr("ledgerFragment", r).log("Checking fragment");
+
try {
verifyLedgerFragment(r, allFragmentsCb, percentageOfLedgerFragmentToBeVerified);
} catch (InvalidFragmentException ife) {
- LOG.error("Invalid fragment found : {}", r);
+ log.error().attr("ledgerFragment", r).log("Invalid fragment found");
allFragmentsCb.operationComplete(
BKException.Code.IncorrectParameterException, r);
} catch (BKException e) {
- LOG.error("BKException when checking fragment : {}", r, e);
+ log.error()
+ .exception(e)
+ .attr("ledgerFragment", r)
+ .log("BKException when checking fragment");
} catch (InterruptedException e) {
- LOG.error("InterruptedException when checking fragment : {}", r, e);
+ log.error()
+ .exception(e)
+ .attr("ledgerFragment", r)
+ .log("InterruptedException when checking fragment");
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
index 5eaeebf6837..e64342bf28c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerCreateOp.java
@@ -30,6 +30,7 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.AsyncCallback.CreateCallback;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.client.BookKeeper.DigestType;
@@ -47,17 +48,14 @@
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Encapsulates asynchronous ledger create operation.
*
*/
+@CustomLog
class LedgerCreateOp {
- static final Logger LOG = LoggerFactory.getLogger(LedgerCreateOp.class);
-
final CreateCallback cb;
LedgerMetadata metadata;
LedgerHandle lh;
@@ -141,18 +139,22 @@ public void initiate() {
break;
} catch (BKNotEnoughBookiesException e) {
if (actualEnsembleSize >= writeQuorumSize + 1) {
- LOG.info("Not enough bookies to create ledger with ensembleSize={},"
- + " writeQuorumSize={} and ackQuorumSize={}, opportusticStriping enabled, try again",
- actualEnsembleSize, writeQuorumSize, ackQuorumSize);
+ log.info()
+ .attr("ensembleSize", actualEnsembleSize)
+ .attr("writeQuorumSize", writeQuorumSize)
+ .attr("ackQuorumSize", ackQuorumSize)
+ .log("Not enough bookies to create ledger with opportusticStriping enabled, try again");
}
lastError = e;
actualEnsembleSize--;
}
}
if (lastError != null) {
- LOG.error("Not enough bookies to create ledger with ensembleSize={},"
- + " writeQuorumSize={} and ackQuorumSize={}",
- actualEnsembleSize, writeQuorumSize, ackQuorumSize);
+ log.error()
+ .attr("ensembleSize", actualEnsembleSize)
+ .attr("writeQuorumSize", writeQuorumSize)
+ .attr("ackQuorumSize", ackQuorumSize)
+ .log("Not enough bookies to create ledger");
createComplete(lastError.getCode(), null);
return;
}
@@ -161,9 +163,11 @@ public void initiate() {
ensemble = bk.getBookieWatcher()
.newEnsemble(actualEnsembleSize, writeQuorumSize, ackQuorumSize, customMetadata);
} catch (BKNotEnoughBookiesException e) {
- LOG.error("Not enough bookies to create ledger with ensembleSize={},"
- + " writeQuorumSize={} and ackQuorumSize={}",
- actualEnsembleSize, writeQuorumSize, ackQuorumSize);
+ log.error()
+ .attr("ensembleSize", actualEnsembleSize)
+ .attr("writeQuorumSize", writeQuorumSize)
+ .attr("ackQuorumSize", ackQuorumSize)
+ .log("Not enough bookies to create ledger");
createComplete(e.getCode(), null);
return;
}
@@ -243,17 +247,26 @@ private void metadataCallback(Versioned writtenMetadata,
lh = new LedgerHandle(bk.getClientCtx(), ledgerId, writtenMetadata, digestType, passwd, writeFlags);
}
} catch (GeneralSecurityException e) {
- LOG.error("Security exception while creating ledger: " + ledgerId, e);
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .exception(e)
+ .log("Security exception while creating ledger");
createComplete(BKException.Code.DigestNotInitializedException, null);
return;
} catch (NumberFormatException e) {
- LOG.error("Incorrectly entered parameter throttle: " + bk.getConf().getThrottleValue(), e);
+ log.error()
+ .exception(e)
+ .attr("throttle", bk.getConf().getThrottleValue())
+ .log("Incorrectly entered parameter throttle");
createComplete(BKException.Code.IncorrectParameterException, null);
return;
}
List curEns = lh.getLedgerMetadata().getEnsembleAt(0L);
- LOG.info("Ensemble: {} for ledger: {}", curEns, lh.getId());
+ log.info()
+ .attr("ensemble", curEns)
+ .attr("ledgerId", lh.getId())
+ .log("New ledger");
for (BookieId bsa : curEns) {
clientStats.getEnsembleBookieDistributionCounter(bsa.toString()).inc();
@@ -344,38 +357,43 @@ public CreateAdvBuilder makeAdv() {
private boolean validate() {
if (builderWriteFlags == null) {
- LOG.error("invalid null writeFlags");
+ log.error("invalid null writeFlags");
return false;
}
if (builderWriteQuorumSize > builderEnsembleSize) {
- LOG.error("invalid writeQuorumSize {} > ensembleSize {}", builderWriteQuorumSize, builderEnsembleSize);
+ log.error()
+ .attr("writeQuorumSize", builderWriteQuorumSize)
+ .attr("ensembleSize", builderEnsembleSize)
+ .log("invalid writeQuorumSize > ensembleSize");
return false;
}
if (builderAckQuorumSize > builderWriteQuorumSize) {
- LOG.error("invalid ackQuorumSize {} > writeQuorumSize {}", builderAckQuorumSize,
- builderWriteQuorumSize);
+ log.error()
+ .attr("ackQuorumSize", builderAckQuorumSize)
+ .attr("writeQuorumSize", builderWriteQuorumSize)
+ .log("invalid ackQuorumSize > writeQuorumSize");
return false;
}
if (builderAckQuorumSize <= 0) {
- LOG.error("invalid ackQuorumSize {} <= 0", builderAckQuorumSize);
+ log.error().attr("ackQuorumSize", builderAckQuorumSize).log("invalid ackQuorumSize <= 0");
return false;
}
if (builderPassword == null) {
- LOG.error("invalid null password");
+ log.error("invalid null password");
return false;
}
if (builderDigestType == null) {
- LOG.error("invalid null digestType");
+ log.error("invalid null digestType");
return false;
}
if (builderCustomMetadata == null) {
- LOG.error("invalid null customMetadata");
+ log.error("invalid null customMetadata");
return false;
}
@@ -441,8 +459,9 @@ private boolean validate() {
return false;
}
if (builderLedgerId != null && builderLedgerId < 0) {
- LOG.error("invalid ledgerId {} < 0. Do not set en explicit value if you want automatic generation",
- builderLedgerId);
+ log.error()
+ .attr("ledgerId", builderLedgerId)
+ .log("invalid ledgerId < 0. Do not set en explicit value if you want automatic generation");
return false;
}
return true;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java
index a19bd761de0..7399d4064ef 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerDeleteOp.java
@@ -24,23 +24,21 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.AsyncCallback.DeleteCallback;
import org.apache.bookkeeper.client.SyncCallbackUtils.SyncDeleteCallback;
import org.apache.bookkeeper.client.api.DeleteBuilder;
import org.apache.bookkeeper.common.util.MathUtils;
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.versioning.Version;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Encapsulates asynchronous ledger delete operation.
*
*/
+@CustomLog
class LedgerDeleteOp {
- static final Logger LOG = LoggerFactory.getLogger(LedgerDeleteOp.class);
-
final BookKeeper bk;
final long ledgerId;
final DeleteCallback cb;
@@ -117,7 +115,7 @@ public CompletableFuture execute() {
private boolean validate() {
if (builderLedgerId == null || builderLedgerId < 0) {
- LOG.error("invalid ledgerId {} < 0", builderLedgerId);
+ log.error().attr("ledgerId", builderLedgerId).log("invalid ledgerId < 0");
return false;
}
return true;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java
index f6d54e1d02b..3f4f74b0352 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragmentReplicator.java
@@ -44,6 +44,7 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiConsumer;
import java.util.stream.Collectors;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
import org.apache.bookkeeper.client.api.WriteFlag;
import org.apache.bookkeeper.common.util.MathUtils;
@@ -60,8 +61,6 @@
import org.apache.bookkeeper.stats.annotations.StatsDoc;
import org.apache.bookkeeper.util.ByteBufList;
import org.apache.zookeeper.AsyncCallback;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This is the helper class for replicating the fragments from one bookie to
@@ -71,6 +70,7 @@
name = REPLICATION_WORKER_SCOPE,
help = "Ledger fragment replicator related stats"
)
+@CustomLog
public class LedgerFragmentReplicator {
// BookKeeper instance
@@ -135,17 +135,15 @@ public LedgerFragmentReplicator(BookKeeper bkc, ClientConfiguration conf) {
this(bkc, NullStatsLogger.INSTANCE, conf);
}
- private static final Logger LOG = LoggerFactory
- .getLogger(LedgerFragmentReplicator.class);
-
private void replicateFragmentInternal(final LedgerHandle lh,
final LedgerFragment lf,
final AsyncCallback.VoidCallback ledgerFragmentMcb,
final Set newBookies,
final BiConsumer onReadEntryFailureCallback) throws InterruptedException {
if (!lf.isClosed()) {
- LOG.error("Trying to replicate an unclosed fragment;"
- + " This is not safe {}", lf);
+ log.error()
+ .attr("ledgerFragment", lf)
+ .log("Trying to replicate an unclosed fragment; This is not safe");
ledgerFragmentMcb.processResult(BKException.Code.UnclosedFragmentException,
null, null);
return;
@@ -158,8 +156,11 @@ private void replicateFragmentInternal(final LedgerHandle lh,
* INVALID_ENTRY_ID and viceversa.
*/
if (startEntryId == INVALID_ENTRY_ID ^ endEntryId == INVALID_ENTRY_ID) {
- LOG.error("For LedgerFragment: {}, seeing inconsistent firstStoredEntryId: {} and lastStoredEntryId: {}",
- lf, startEntryId, endEntryId);
+ log.error()
+ .attr("ledgerFragment", lf)
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .log("For LedgerFragment, seeing inconsistent firstStoredEntryId and lastStoredEntryId");
assert false;
}
@@ -235,8 +236,10 @@ void replicate(final LedgerHandle lh, final LedgerFragment lf,
throws InterruptedException {
Set partitionedFragments = splitIntoSubFragments(lh, lf,
bkc.getConf().getRereplicationEntryBatchSize());
- LOG.info("Replicating fragment {} in {} sub fragments.",
- lf, partitionedFragments.size());
+ log.info()
+ .attr("ledgerFragment", lf)
+ .attr("size", partitionedFragments.size())
+ .log("Replicating fragment in sub fragments.");
replicateNextBatch(lh, partitionedFragments.iterator(),
ledgerFragmentMcb, targetBookieAddresses, onReadEntryFailureCallback);
}
@@ -300,8 +303,11 @@ static Set splitIntoSubFragments(LedgerHandle lh,
* INVALID_ENTRY_ID and viceversa.
*/
if (firstEntryId == INVALID_ENTRY_ID ^ lastEntryId == INVALID_ENTRY_ID) {
- LOG.error("For LedgerFragment: {}, seeing inconsistent firstStoredEntryId: {} and lastStoredEntryId: {}",
- ledgerFragment, firstEntryId, lastEntryId);
+ log.error()
+ .attr("ledgerFragment", ledgerFragment)
+ .attr("firstEntryId", firstEntryId)
+ .attr("lastEntryId", lastEntryId)
+ .log("For LedgerFragment, seeing inconsistent firstStoredEntryId and lastStoredEntryId");
assert false;
}
@@ -365,8 +371,12 @@ void recoverLedgerFragmentEntry(final Long entryId,
@Override
public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("BK error writing entry for ledgerId: {}, entryId: {}, bookie: {}",
- ledgerId, entryId, addr, BKException.create(rc));
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .exceptionMessage(BKException.create(rc))
+ .log("BK error writing entry");
if (completed.compareAndSet(false, true)) {
ledgerFragmentEntryMcb.processResult(rc, null, null);
}
@@ -375,10 +385,13 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob
if (ctx instanceof Long) {
numBytesWritten.registerSuccessfulValue((Long) ctx);
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Success writing ledger id {}, entry id {} to a new bookie {}!",
- ledgerId, entryId, addr);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .log("Success writing entry to a new bookie");
+
if (numCompleted.incrementAndGet() == newBookies.size() && completed.compareAndSet(false, true)) {
ledgerFragmentEntryMcb.processResult(rc, null, null);
}
@@ -397,8 +410,11 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob
public void readComplete(int rc, LedgerHandle lh,
Enumeration seq, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("BK error reading ledger entry: " + entryId,
- BKException.create(rc));
+ log.error()
+ .attr("ledgerId", lh.getId())
+ .attr("entryId", entryId)
+ .exceptionMessage(BKException.create(rc))
+ .log("BK error reading ledger entry");
onReadEntryFailureCallback.accept(ledgerId, entryId);
ledgerFragmentEntryMcb.processResult(rc, null, null);
return;
@@ -466,8 +482,12 @@ void batchRecoverLedgerFragmentEntry(final long startEntryId,
@Override
public void readComplete(int rc, LedgerHandle lh, Enumeration seq, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("BK error reading ledger entries: {} - {}",
- startEntryId, endEntryId, BKException.create(rc));
+ log.error()
+ .attr("ledgerId", lh.getId())
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .exceptionMessage(BKException.create(rc))
+ .log("BK error reading ledger entries");
onReadEntryFailureCallback.accept(lh.getId(), startEntryId);
for (int i = 0; i < entriesToReplicateCnt; i++) {
ledgerFragmentMcb.processResult(rc, null, null);
@@ -503,8 +523,12 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq,
@Override
public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("BK error writing entry for ledgerId: {}, entryId: {}, bookie: {}",
- ledgerId, entryId, addr, BKException.create(rc));
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .attr("create", BKException.create(rc))
+ .log("BK error writing entry");
if (completed.compareAndSet(false, true)) {
ledgerFragmentMcb.processResult(rc, null, null);
}
@@ -513,10 +537,13 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Ob
if (ctx instanceof Long) {
numBytesWritten.registerSuccessfulValue((Long) ctx);
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Success writing ledger id {}, entry id {} to a new bookie {}!",
- ledgerId, entryId, addr);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .log("Success writing entry to a new bookie");
+
if (numCompleted.incrementAndGet() == newBookies.size()
&& completed.compareAndSet(false, true)) {
ledgerFragmentMcb.processResult(rc, null, null);
@@ -582,8 +609,10 @@ static class SingleFragmentCallback implements AsyncCallback.VoidCallback {
@Override
public void processResult(int rc, String path, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("BK error replicating ledger fragments for ledger: "
- + lh.getId(), BKException.create(rc));
+ log.error()
+ .attr("create", BKException.create(rc))
+ .attr("ledgerId", lh.getId())
+ .log("BK error replicating ledger fragments");
ledgerFragmentsMcb.processResult(rc, null, null);
return;
}
@@ -619,12 +648,16 @@ private static void updateEnsembleInfo(
updateLoop.run().whenComplete((result, ex) -> {
if (ex == null) {
- LOG.info("Updated ZK to point ledger fragments"
- + " from old bookies to new bookies: {}", oldBookie2NewBookie);
+ log.info()
+ .attr("oldBookie2NewBookie", oldBookie2NewBookie)
+ .log("Updated ZK to point ledger fragments from old bookies to new bookies");
ensembleUpdatedCb.processResult(BKException.Code.OK, null, null);
} else {
- LOG.error("Error updating ledger config metadata for ledgerId {}", lh.getId(), ex);
+ log.error()
+ .exception(ex)
+ .attr("ledgerId", lh.getId())
+ .log("Error updating ledger config metadata for ledgerId");
ensembleUpdatedCb.processResult(
BKException.getExceptionCode(ex, BKException.Code.UnexpectedConditionException),
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
index 7d1f67a2e4b..9714803d5df 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
@@ -31,6 +31,7 @@
import com.google.common.collect.Iterators;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.RateLimiter;
+import io.github.merlimat.slog.Logger;
import io.netty.buffer.ByteBuf;
import io.netty.buffer.Unpooled;
import java.security.GeneralSecurityException;
@@ -85,15 +86,14 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.versioning.Versioned;
import org.apache.commons.collections4.IteratorUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Ledger handle contains ledger metadata and is used to access the read and
* write operations to a ledger.
*/
public class LedgerHandle implements WriteHandle {
- static final Logger LOG = LoggerFactory.getLogger(LedgerHandle.class);
+
+ protected final Logger log;
private static final int STICKY_READ_BOOKIE_INDEX_UNSET = -1;
@@ -180,6 +180,7 @@ private enum HandleState {
BookKeeper.DigestType digestType, byte[] password,
EnumSet writeFlags)
throws GeneralSecurityException, NumberFormatException {
+ this.log = Logger.get(LedgerHandle.class).with().attr("ledgerId", ledgerId).build();
this.clientCtx = clientCtx;
this.versionedMetadata = versionedMetadata;
@@ -516,9 +517,9 @@ void asyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc)
try {
doAsyncCloseInternal(cb, ctx, rc);
} catch (RejectedExecutionException re) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Failed to close ledger {} : ", ledgerId, re);
- }
+
+ log.debug().exception(re).log("Failed to close ledger");
+
errorOutPendingAdds(BookKeeper.getReturnRc(clientCtx.getBookieClient(), rc));
cb.closeComplete(BookKeeper.getReturnRc(clientCtx.getBookieClient(), BKException.Code.InterruptedException),
this, ctx);
@@ -573,10 +574,12 @@ void doAsyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc
}
if (prevHandleState != HandleState.CLOSED) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Closing ledger: {} at entryId {} with {} bytes", getId(), lastEntry,
- finalLength);
- }
+
+ log.debug()
+ .attr("lastEntry", lastEntry)
+ .attr("finalLength", finalLength)
+ .log("Closing ledger");
+
tearDownWriteHandleState();
new MetadataUpdateLoop(
@@ -591,14 +594,14 @@ void doAsyncCloseInternal(final CloseCallback cb, final Object ctx, final int rc
&& finalLength == metadata.getLength()) {
return false;
} else {
- LOG.error("Metadata conflict when closing ledger {}."
- + " Another client may have recovered the ledger while "
- + "there"
- + " were writes outstanding. (local lastEntry:{} "
- + "length:{}) "
- + " (metadata lastEntry:{} length:{})",
- getId(), lastEntry, finalLength,
- metadata.getLastEntryId(), metadata.getLength());
+ log.error()
+ .attr("lastEntryId", lastEntry)
+ .attr("length", finalLength)
+ .attr("metadataLastEntryId", metadata.getLastEntryId())
+ .attr("metadataLength", metadata.getLength())
+ .log("Metadata conflict when closing ledger."
+ + " Another client may have recovered the ledger while"
+ + " there were writes outstanding.");
throw new BKException.BKMetadataVersionException();
}
} else {
@@ -721,15 +724,20 @@ public Enumeration batchReadUnconfirmedEntries(long firstEntry, int
public void asyncReadEntries(long firstEntry, long lastEntry, ReadCallback cb, Object ctx) {
// Little sanity check
if (firstEntry < 0 || firstEntry > lastEntry) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} lastEntry:{}",
- ledgerId, firstEntry, lastEntry);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .log("IncorrectParameterException");
cb.readComplete(BKException.Code.IncorrectParameterException, this, null, ctx);
return;
}
if (lastEntry > lastAddConfirmed) {
- LOG.error("ReadEntries exception on ledgerId:{} firstEntry:{} lastEntry:{} lastAddConfirmed:{}",
- ledgerId, firstEntry, lastEntry, lastAddConfirmed);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .log("ReadEntries exception");
cb.readComplete(BKException.Code.ReadException, this, null, ctx);
return;
}
@@ -755,8 +763,10 @@ public void asyncReadEntries(long firstEntry, long lastEntry, ReadCallback cb, O
public void asyncBatchReadEntries(long startEntry, int maxCount, long maxSize, ReadCallback cb, Object ctx) {
// Little sanity check
if (startEntry < 0 || startEntry > lastAddConfirmed) {
- LOG.error("IncorrectParameterException on ledgerId:{} startEntry:{} lastAddConfirmed:{}",
- ledgerId, startEntry, lastAddConfirmed);
+ log.error()
+ .attr("startEntry", startEntry)
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .log("IncorrectParameterException");
cb.readComplete(BKException.Code.IncorrectParameterException, this, null, ctx);
return;
}
@@ -811,8 +821,10 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq,
public void asyncReadUnconfirmedEntries(long firstEntry, long lastEntry, ReadCallback cb, Object ctx) {
// Little sanity check
if (firstEntry < 0 || firstEntry > lastEntry) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} lastEntry:{}",
- ledgerId, firstEntry, lastEntry);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .log("IncorrectParameterException");
cb.readComplete(BKException.Code.IncorrectParameterException, this, null, ctx);
return;
}
@@ -838,7 +850,9 @@ public void asyncBatchReadUnconfirmedEntries(long startEntry, int maxCount, long
Object ctx) {
// Little sanity check
if (startEntry < 0) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{}", ledgerId, startEntry);
+ log.error()
+ .attr("startEntry", startEntry)
+ .log("IncorrectParameterException");
cb.readComplete(BKException.Code.IncorrectParameterException, this, null, ctx);
}
if (notSupportBatchRead()) {
@@ -874,14 +888,19 @@ public void readComplete(int rc, LedgerHandle lh, Enumeration seq,
public CompletableFuture readAsync(long firstEntry, long lastEntry) {
// Little sanity check
if (firstEntry < 0 || firstEntry > lastEntry) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} lastEntry:{}",
- ledgerId, firstEntry, lastEntry);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .log("IncorrectParameterException");
return FutureUtils.exception(new BKIncorrectParameterException());
}
if (lastEntry > lastAddConfirmed) {
- LOG.error("ReadAsync exception on ledgerId:{} firstEntry:{} lastEntry:{} lastAddConfirmed:{}",
- ledgerId, firstEntry, lastEntry, lastAddConfirmed);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .log("ReadAsync exception");
return FutureUtils.exception(new BKReadException());
}
@@ -903,12 +922,16 @@ public CompletableFuture readAsync(long firstEntry, long lastEntr
public CompletableFuture batchReadAsync(long startEntry, int maxCount, long maxSize) {
// Little sanity check
if (startEntry < 0) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{}", ledgerId, startEntry);
+ log.error()
+ .attr("startEntry", startEntry)
+ .log("IncorrectParameterException");
return FutureUtils.exception(new BKIncorrectParameterException());
}
if (startEntry > lastAddConfirmed) {
- LOG.error("ReadAsync exception on ledgerId:{} firstEntry:{} lastAddConfirmed:{}",
- ledgerId, startEntry, lastAddConfirmed);
+ log.error()
+ .attr("startEntry", startEntry)
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .log("ReadAsync exception");
return FutureUtils.exception(new BKReadException());
}
if (notSupportBatchRead()) {
@@ -956,17 +979,19 @@ private CompletableFuture batchReadEntriesInternalAsync(long star
boolean isRecoveryRead) {
int nettyMaxFrameSizeBytes = clientCtx.getConf().nettyMaxFrameSizeBytes;
if (maxSize > nettyMaxFrameSizeBytes) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("The max size is greater than nettyMaxFrameSizeBytes, "
- + "use nettyMaxFrameSizeBytes:{} to replace it.", nettyMaxFrameSizeBytes);
- }
+
+ log.debug()
+ .attr("nettyMaxFrameSizeBytes", nettyMaxFrameSizeBytes)
+ .log("The max size is greater than nettyMaxFrameSizeBytes, use nettyMaxFrameSizeBytes: to replace it.");
+
maxSize = nettyMaxFrameSizeBytes;
}
if (maxSize <= 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("The max size is negative, use nettyMaxFrameSizeBytes:{} to replace it.",
- nettyMaxFrameSizeBytes);
- }
+
+ log.debug()
+ .attr("nettyMaxFrameSizeBytes", nettyMaxFrameSizeBytes)
+ .log("The max size is negative, use nettyMaxFrameSizeBytes to replace it.");
+
maxSize = nettyMaxFrameSizeBytes;
}
BatchedReadOp op = new BatchedReadOp(this, clientCtx,
@@ -1036,8 +1061,10 @@ private CompletableFuture batchReadEntriesInternalAsync(long star
public CompletableFuture readUnconfirmedAsync(long firstEntry, long lastEntry) {
// Little sanity check
if (firstEntry < 0 || firstEntry > lastEntry) {
- LOG.error("IncorrectParameterException on ledgerId:{} firstEntry:{} lastEntry:{}",
- ledgerId, firstEntry, lastEntry);
+ log.error()
+ .attr("firstEntry", firstEntry)
+ .attr("lastEntry", lastEntry)
+ .log("IncorrectParameterException");
return FutureUtils.exception(new BKIncorrectParameterException());
}
@@ -1223,7 +1250,7 @@ public CompletableFuture appendAsync(ByteBuf data) {
* @return the entryId of the new inserted entry
*/
public long addEntry(final long entryId, byte[] data) throws InterruptedException, BKException {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv interface.");
throw BKException.create(BKException.Code.IllegalOpException);
}
@@ -1241,9 +1268,9 @@ public long addEntry(final long entryId, byte[] data) throws InterruptedExceptio
*/
public long addEntry(byte[] data, int offset, int length)
throws InterruptedException, BKException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Adding entry {}", data);
- }
+
+ log.debug().attr("data", data).log("Adding entry");
+
SyncAddCallback callback = new SyncAddCallback();
asyncAddEntry(data, offset, length, callback, null);
@@ -1269,7 +1296,7 @@ public long addEntry(byte[] data, int offset, int length)
*/
public long addEntry(final long entryId, byte[] data, int offset, int length) throws InterruptedException,
BKException {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
throw BKException.create(BKException.Code.IllegalOpException);
}
@@ -1305,7 +1332,7 @@ public void asyncAddEntry(final byte[] data, final AddCallback cb,
* some control object
*/
public void asyncAddEntry(final long entryId, final byte[] data, final AddCallback cb, final Object ctx) {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
cb.addCompleteWithLatency(BKException.Code.IllegalOpException, LedgerHandle.this, entryId, 0, ctx);
}
@@ -1367,7 +1394,7 @@ public void asyncAddEntry(ByteBuf data, final AddCallback cb, final Object ctx)
*/
public void asyncAddEntry(final long entryId, final byte[] data, final int offset, final int length,
final AddCallback cb, final Object ctx) {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
cb.addCompleteWithLatency(BKException.Code.IllegalOpException, LedgerHandle.this, entryId, 0, ctx);
}
@@ -1393,7 +1420,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse
*/
public void asyncAddEntry(final long entryId, final byte[] data, final int offset, final int length,
final AddCallbackWithLatency cb, final Object ctx) {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
cb.addCompleteWithLatency(BKException.Code.IllegalOpException, LedgerHandle.this, entryId, 0, ctx);
}
@@ -1414,7 +1441,7 @@ public void asyncAddEntry(final long entryId, final byte[] data, final int offse
*/
public void asyncAddEntry(final long entryId, ByteBuf data,
final AddCallbackWithLatency cb, final Object ctx) {
- LOG.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
+ log.error("To use this feature Ledger must be created with createLedgerAdv() interface.");
cb.addCompleteWithLatency(BKException.Code.IllegalOpException, LedgerHandle.this, entryId, 0, ctx);
}
@@ -1441,7 +1468,7 @@ public CompletableFuture force() {
executeOrdered(new Runnable() {
@Override
public void run() {
- LOG.warn("Force() attempted on a closed ledger: {}", ledgerId);
+ log.warn("Force() attempted on a closed ledger");
result.completeExceptionally(new BKException.BKLedgerClosedException());
}
@@ -1566,10 +1593,11 @@ protected boolean waitForWritable(DistributionSchedule.WriteSet writeSet,
}
}
if (backoff > 1) {
- LOG.info("Spent {} ms waiting for {} writable channels, writable result {}",
- MathUtils.elapsedMSec(startTime),
- writeSet.size() - allowedNonWritableCount,
- writableResult);
+ log.info()
+ .attr("elapsedMSec", MathUtils.elapsedMSec(startTime))
+ .attr("writableChannels", writeSet.size() - allowedNonWritableCount)
+ .attr("writableResult", writableResult)
+ .log("Spent ms waiting for writable channels");
}
}
@@ -1610,7 +1638,7 @@ protected void doAsyncAddEntry(final PendingAddOp op) {
executeOrdered(new Runnable() {
@Override
public void run() {
- LOG.warn("Attempt to add to closed ledger: {}", ledgerId);
+ log.warn("Attempt to add to closed ledger");
op.cb.addCompleteWithLatency(BKException.Code.LedgerClosedException,
LedgerHandle.this, INVALID_ENTRY_ID, 0, op.ctx);
op.recyclePendAddOpObject();
@@ -2053,7 +2081,9 @@ void handleUnrecoverableErrorDuringAdd(int rc) {
errorOutPendingAdds(rc);
return;
}
- LOG.error("Closing ledger {} due to {}", ledgerId, BKException.codeLogger(rc));
+ log.error()
+ .attr("rc", BKException.codeLogger(rc))
+ .log("Closing ledger due to error");
asyncCloseInternal(NoopCloseCallback.instance, null, rc);
}
@@ -2065,7 +2095,7 @@ private void monitorPendingAddOps() {
}
}
if (timedOut > 0) {
- LOG.info("Timed out {} add ops", timedOut);
+ log.info().attr("timedOutOpsCount", timedOut).log("Timed out add ops");
}
}
@@ -2097,17 +2127,19 @@ void sendAddSuccessCallbacks() {
while ((pendingAddOp = pendingAddOps.peek()) != null
&& !changingEnsemble) {
if (!pendingAddOp.completed) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("pending add not completed: {}", pendingAddOp);
- }
+
+ log.debug().attr("pendingAddOp", pendingAddOp).log("pending add not completed");
+
return;
}
// Check if it is the next entry in the sequence.
if (pendingAddOp.entryId != 0 && pendingAddOp.entryId != pendingAddsSequenceHead + 1) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Head of the queue entryId: {} is not the expected value: {}", pendingAddOp.entryId,
- pendingAddsSequenceHead + 1);
- }
+
+ log.debug()
+ .attr("entryId", pendingAddOp.entryId)
+ .attr("expectedEntryId", pendingAddsSequenceHead + 1)
+ .log("Head of the queue is not the expected entryId");
+
return;
}
@@ -2157,21 +2189,23 @@ void maybeHandleDelayedWriteBookieFailure() {
void handleBookieFailure(final Map failedBookies) {
if (clientCtx.getConf().disableEnsembleChangeFeature.isAvailable()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Ensemble change is disabled. Retry sending to failed bookies {} for ledger {}.",
- failedBookies, ledgerId);
- }
+
+ log.debug()
+ .attr("failedBookies", failedBookies)
+ .log("Ensemble change is disabled. Retry sending to failed bookies");
+
executeOrdered(() ->
unsetSuccessAndSendWriteRequest(getCurrentEnsemble(), failedBookies.keySet()));
return;
}
if (writeFlags.contains(WriteFlag.DEFERRED_SYNC)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cannot perform ensemble change with write flags {}. "
- + "Failed bookies {} for ledger {}.",
- writeFlags, failedBookies, ledgerId);
- }
+
+ log.debug()
+ .attr("writeFlags", writeFlags)
+ .attr("failedBookies", failedBookies)
+ .log("Cannot perform ensemble change with write flag on failed bookies");
+
handleUnrecoverableErrorDuringAdd(WriteException);
return;
}
@@ -2202,19 +2236,24 @@ void handleBookieFailure(final Map failedBookies) {
void ensembleChangeLoop(List origEnsemble, Map failedBookies) {
int ensembleChangeId = numEnsembleChanges.incrementAndGet();
ensembleChangeCounter.inc();
- String logContext = String.format("[EnsembleChange(ledger:%d, change-id:%010d)]", ledgerId, ensembleChangeId);
// when the ensemble changes are too frequent, close handle
if (ensembleChangeId > clientCtx.getConf().maxAllowedEnsembleChanges) {
- LOG.info("{} reaches max allowed ensemble change number {}",
- logContext, clientCtx.getConf().maxAllowedEnsembleChanges);
+ log.info()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("maxAllowedEnsembleChanges", clientCtx.getConf().maxAllowedEnsembleChanges)
+ .log("reaches max allowed ensemble change number");
handleUnrecoverableErrorDuringAdd(WriteException);
return;
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} Replacing {} in {}", logContext, failedBookies, origEnsemble);
- }
+
+ log.debug()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("failedBookies", failedBookies)
+ .attr("origEnsemble", origEnsemble)
+ .log("Replacing bookies");
+
AtomicInteger attempts = new AtomicInteger(0);
new MetadataUpdateLoop(
@@ -2229,16 +2268,21 @@ void ensembleChangeLoop(List origEnsemble, Map fail
List currentEnsemble = getCurrentEnsemble();
List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
- clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext);
+ clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, ensembleChangeId);
Long lastEnsembleKey = LedgerMetadataUtils.getLastEnsembleKey(metadata);
LedgerMetadataBuilder builder = LedgerMetadataBuilder.from(metadata);
long newEnsembleStartEntry = getLastAddConfirmed() + 1;
checkState(lastEnsembleKey <= newEnsembleStartEntry,
"New ensemble must either replace the last ensemble, or add a new one");
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}[attempt:{}] changing ensemble from: {} to: {} starting at entry: {}",
- logContext, attempts.get(), currentEnsemble, newEnsemble, newEnsembleStartEntry);
- }
+
+ log.debug()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("attempts", attempts.get())
+ .attr("currentEnsemble", currentEnsemble)
+ .attr("newEnsemble", newEnsemble)
+ .attr("newEnsembleStartEntry", newEnsembleStartEntry)
+ .log("Changing ensemble");
+
if (lastEnsembleKey.equals(newEnsembleStartEntry)) {
return builder.replaceEnsembleEntry(newEnsembleStartEntry, newEnsemble).build();
@@ -2249,25 +2293,33 @@ void ensembleChangeLoop(List origEnsemble, Map fail
this::setLedgerMetadata)
.run().whenCompleteAsync((metadata, ex) -> {
if (ex != null) {
- LOG.warn("{}[attempt:{}] Exception changing ensemble", logContext, attempts.get(), ex);
+ log.warn()
+ .exception(ex)
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("attempts", attempts.get())
+ .log("Exception changing ensemble");
handleUnrecoverableErrorDuringAdd(BKException.getExceptionCode(ex, WriteException));
} else if (metadata.getValue().isClosed()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}[attempt:{}] Metadata closed during attempt to replace bookie."
- + " Another client must have recovered the ledger.", logContext, attempts.get());
- }
+ log.debug()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("attempts", attempts.get())
+ .log("Metadata closed during attempt to replace bookie."
+ + " Another client must have recovered the ledger.");
+
handleUnrecoverableErrorDuringAdd(BKException.Code.LedgerClosedException);
} else if (metadata.getValue().getState() == LedgerMetadata.State.IN_RECOVERY) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}[attempt:{}] Metadata marked as in-recovery during attempt to replace bookie."
- + " Another client must be recovering the ledger.", logContext, attempts.get());
- }
+ log.debug()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("attempts", attempts.get())
+ .log("Metadata marked as in-recovery during attempt to replace bookie."
+ + " Another client must be recovering the ledger.");
handleUnrecoverableErrorDuringAdd(BKException.Code.LedgerFencedException);
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{}[attempt:{}] Success updating metadata.", logContext, attempts.get());
- }
+ log.debug()
+ .attr("ensembleChangeId", ensembleChangeId)
+ .attr("get", attempts.get())
+ .log("Success updating metadata");
List newEnsemble = null;
Set replaced = null;
@@ -2280,7 +2332,9 @@ void ensembleChangeLoop(List origEnsemble, Map fail
} else {
newEnsemble = getCurrentEnsemble();
replaced = EnsembleUtils.diffEnsemble(origEnsemble, newEnsemble);
- LOG.info("New Ensemble: {} for ledger: {}", newEnsemble, ledgerId);
+ log.info()
+ .attr("newEnsemble", newEnsemble)
+ .log("New Ensemble");
changingEnsemble = false;
}
@@ -2312,12 +2366,13 @@ void registerOperationFailureOnBookie(BookieId bookie, long entryId) {
}
static class NoopCloseCallback implements CloseCallback {
+ private static final Logger noopLog = Logger.get(NoopCloseCallback.class);
static NoopCloseCallback instance = new NoopCloseCallback();
@Override
public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.warn("Close failed: {}", BKException.codeLogger(rc));
+ noopLog.warn().attr("codeLogger", BKException.codeLogger(rc)).log("Close failed");
}
// noop
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
index c94a9154f51..215d162827b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandleAdv.java
@@ -38,17 +38,12 @@
import org.apache.bookkeeper.client.api.WriteAdvHandle;
import org.apache.bookkeeper.client.api.WriteFlag;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
/**
* Ledger Advanced handle extends {@link LedgerHandle} to provide API to add entries with
* user supplied entryIds. Through this interface Ledger Length may not be accurate while the
* ledger being written.
*/
public class LedgerHandleAdv extends LedgerHandle implements WriteAdvHandle {
- static final Logger LOG = LoggerFactory.getLogger(LedgerHandleAdv.class);
static class PendingOpsComparator implements Comparator, Serializable {
@Override
@@ -101,9 +96,9 @@ public long addEntry(final long entryId, byte[] data) throws InterruptedExceptio
@Override
public long addEntry(final long entryId, byte[] data, int offset, int length) throws InterruptedException,
BKException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Adding entry {}", data);
- }
+
+ log.debug().attr("data", data).log("Adding entry");
+
SyncAddCallback callback = new SyncAddCallback();
asyncAddEntry(entryId, data, offset, length, callback, null);
@@ -207,7 +202,7 @@ public void asyncAddEntry(final long entryId, ByteBuf data,
op.setEntryId(entryId);
if ((entryId <= this.lastAddConfirmed) || pendingAddOps.contains(op)) {
- LOG.error("Trying to re-add duplicate entryid:{}", entryId);
+ log.error().attr("entryId", entryId).log("Trying to re-add duplicate entryId");
op.submitCallback(BKException.Code.DuplicateEntryIdException);
return;
}
@@ -245,7 +240,7 @@ protected void doAsyncAddEntry(final PendingAddOp op) {
clientCtx.getMainWorkerPool().submit(new Runnable() {
@Override
public void run() {
- LOG.warn("Attempt to add to closed ledger: {}", ledgerId);
+ log.warn("Attempt to add to closed ledger");
op.cb.addCompleteWithLatency(BKException.Code.LedgerClosedException,
LedgerHandleAdv.this, op.getEntryId(), 0, op.ctx);
op.recyclePendAddOpObject();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
index 14a17bb5fd7..e1adcbd37c6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataImpl.java
@@ -31,13 +31,12 @@
import java.util.Optional;
import java.util.TreeMap;
import java.util.stream.Collectors;
+import lombok.CustomLog;
import lombok.EqualsAndHashCode;
import org.apache.bookkeeper.client.api.DigestType;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.client.api.LedgerMetadata.State;
import org.apache.bookkeeper.net.BookieId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This class encapsulates all the ledger metadata that is persistently stored
@@ -48,8 +47,8 @@
@EqualsAndHashCode(exclude =
"ledgerId" // ledgerId is not serialized inside ZK node data
)
+@CustomLog
class LedgerMetadataImpl implements LedgerMetadata {
- static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataImpl.class);
private final long ledgerId;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java
index ab26be7d65f..1a24e8454dc 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadataUtils.java
@@ -23,16 +23,15 @@
import java.util.List;
import java.util.Set;
import java.util.SortedMap;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.net.BookieId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Utilities for working with ledger metadata.
*/
+@CustomLog
public class LedgerMetadataUtils {
- static final Logger LOG = LoggerFactory.getLogger(LedgerMetadataUtils.class);
static List getCurrentEnsemble(LedgerMetadata metadata) {
return getLastEnsembleValue(metadata);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
index 52c8190e1f2..6e0fe9b6c03 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerOpenOp.java
@@ -23,6 +23,7 @@
import static org.apache.bookkeeper.client.BookKeeper.DigestType.fromApiDigestType;
+import io.github.merlimat.slog.Logger;
import java.security.GeneralSecurityException;
import java.util.Arrays;
import java.util.concurrent.CompletableFuture;
@@ -40,15 +41,14 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.util.OrderedGenericCallback;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Encapsulates the ledger open operation.
*
*/
class LedgerOpenOp {
- static final Logger LOG = LoggerFactory.getLogger(LedgerOpenOp.class);
+
+ private final Logger log;
final BookKeeper bk;
final long ledgerId;
@@ -89,6 +89,7 @@ class LedgerOpenOp {
public LedgerOpenOp(BookKeeper bk, BookKeeperClientStats clientStats,
long ledgerId, DigestType digestType, byte[] passwd,
OpenCallback cb, Object ctx) {
+ this.log = Logger.get(LedgerOpenOp.class).with().attr("ledgerId", ledgerId).build();
this.bk = bk;
this.ledgerId = ledgerId;
this.passwd = passwd;
@@ -101,6 +102,7 @@ public LedgerOpenOp(BookKeeper bk, BookKeeperClientStats clientStats,
public LedgerOpenOp(BookKeeper bk, BookKeeperClientStats clientStats,
long ledgerId, OpenCallback cb, Object ctx) {
+ this.log = Logger.get(LedgerOpenOp.class).with().attr("ledgerId", ledgerId).build();
this.bk = bk;
this.ledgerId = ledgerId;
this.cb = cb;
@@ -180,14 +182,14 @@ private void openWithMetadata(Versioned versionedMetadata) {
if (metadata.hasPassword()) {
if (!Arrays.equals(passwd, metadata.getPassword())) {
- LOG.error("Provided passwd does not match that in metadata");
+ log.error("Provided passwd does not match that in metadata");
openComplete(BKException.Code.UnauthorizedAccessException, null);
return;
}
// if `digest auto detection` is enabled, ignore the suggested digest type, this allows digest type
// changes. e.g. moving from `crc32` to `crc32c`.
if (suggestedDigestType != fromApiDigestType(metadata.getDigestType()) && !enableDigestAutodetection) {
- LOG.error("Provided digest does not match that in metadata");
+ log.error("Provided digest does not match that in metadata");
openComplete(BKException.Code.DigestMatchException, null);
return;
}
@@ -215,11 +217,12 @@ private void openWithMetadata(Versioned versionedMetadata) {
lh = new ReadOnlyLedgerHandle(bk.getClientCtx(), ledgerId, versionedMetadata, digestType,
passwd, watchImmediately);
} catch (GeneralSecurityException e) {
- LOG.error("Security exception while opening ledger: " + ledgerId, e);
+ log.error().exception(e).attr("ledgerId", ledgerId).log("Security exception while opening ledger");
openComplete(BKException.Code.DigestNotInitializedException, null);
return;
} catch (NumberFormatException e) {
- LOG.error("Incorrectly entered parameter throttle: " + bk.getConf().getThrottleValue(), e);
+ log.error().exception(e).attr("throttle", bk.getConf().getThrottleValue())
+ .log("Incorrectly entered parameter throttle");
openComplete(BKException.Code.IncorrectParameterException, null);
return;
}
@@ -242,7 +245,9 @@ public void safeOperationComplete(int rc, Void result) {
} else {
closeLedgerHandleAsync().whenComplete((ignore, ex) -> {
if (ex != null) {
- LOG.error("Ledger {} close failed", ledgerId, ex);
+ log.error()
+ .exception(ex)
+ .log("Ledger close failed");
}
if (rc == BKException.Code.UnauthorizedAccessException
|| rc == BKException.Code.TimeoutException) {
@@ -266,14 +271,18 @@ public void readLastConfirmedComplete(int rc,
if (rc == BKException.Code.TimeoutException) {
closeLedgerHandleAsync().whenComplete((r, ex) -> {
if (ex != null) {
- LOG.error("Ledger {} close failed", ledgerId, ex);
+ log.error()
+ .exception(ex)
+ .log("Ledger close failed");
}
openComplete(bk.getReturnRc(rc), null);
});
} else if (rc != BKException.Code.OK) {
closeLedgerHandleAsync().whenComplete((r, ex) -> {
if (ex != null) {
- LOG.error("Ledger {} close failed", ledgerId, ex);
+ log.error()
+ .exception(ex)
+ .log("Ledger close failed");
}
openComplete(bk.getReturnRc(BKException.Code.ReadException), null);
});
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
index 16a95cbc895..52c30aeec1d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
@@ -18,14 +18,13 @@
package org.apache.bookkeeper.client;
import com.google.common.annotations.VisibleForTesting;
+import io.github.merlimat.slog.Logger;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This class encapsulated the ledger recovery operation. It first does a read
@@ -35,7 +34,7 @@
*/
class LedgerRecoveryOp implements ReadEntryListener, AddCallback {
- static final Logger LOG = LoggerFactory.getLogger(LedgerRecoveryOp.class);
+ private final Logger log;
final LedgerHandle lh;
final ClientContext clientCtx;
@@ -70,6 +69,7 @@ protected LedgerMetadata getLedgerMetadata() {
}
public LedgerRecoveryOp(LedgerHandle lh, ClientContext clientCtx) {
+ this.log = Logger.get(LedgerRecoveryOp.class).with().attr("ledgerId", lh.getId()).build();
readCount = new AtomicLong(0);
writeCount = new AtomicLong(0);
readDone = false;
@@ -195,8 +195,10 @@ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object c
lh.length.set(entry.getLength() - (long) data.length);
// check whether entry id is expected, so we won't overwritten any entries by mistake
if (entry.getEntryId() != lh.lastAddPushed + 1) {
- LOG.error("Unexpected to recovery add entry {} as entry {} for ledger {}.",
- entry.getEntryId(), (lh.lastAddPushed + 1), lh.getId());
+ log.error()
+ .attr("entryId", entry.getEntryId())
+ .attr("expectedEntryId", (lh.lastAddPushed + 1))
+ .log("Unexpected entryId during recovery add");
rc = BKException.Code.UnexpectedConditionException;
}
}
@@ -221,14 +223,21 @@ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object c
// otherwise, some other error, we can't handle
if (BKException.Code.OK != rc && !promise.isDone()) {
- LOG.error("Failure {} while reading entries: ({} - {}), ledger: {} while recovering ledger",
- BKException.getMessage(rc), startEntryToRead, endEntryToRead, lh.getId());
+ log.error()
+ .attr("getMessage", BKException.getMessage(rc))
+ .attr("startEntryId", startEntryToRead)
+ .attr("endEntryId", endEntryToRead)
+ .attr("ledgerId", lh.getId())
+ .log("Failure while reading entries: ( - ), ledger: while recovering ledger");
submitCallback(rc);
} else if (BKException.Code.OK == rc) {
// we are here is because we successfully read an entry but readDone was already set to true.
// this would happen on recovery a ledger than has gaps in the tail.
- LOG.warn("Successfully read entry {} for ledger {}, but readDone is already {}",
- entry.getEntryId(), lh.getId(), readDone);
+ log.warn()
+ .attr("entryId", entry.getEntryId())
+ .attr("ledgerId", lh.getId())
+ .attr("readDone", readDone)
+ .log("Successfully read entry, but readDone is already set");
}
return;
}
@@ -236,8 +245,11 @@ public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object c
@Override
public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
if (rc != BKException.Code.OK) {
- LOG.error("Failure {} while writing entry: {} while recovering ledger: {}",
- BKException.codeLogger(rc), entryId + 1, lh.ledgerId);
+ log.error()
+ .attr("codeLogger", BKException.codeLogger(rc))
+ .attr("entryId", entryId + 1)
+ .attr("ledgerId", lh.ledgerId)
+ .log("Failure while writing entry while recovering ledger");
submitCallback(rc);
return;
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
index 18af8d9188b..f563c458c1c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
@@ -21,11 +21,11 @@
package org.apache.bookkeeper.client;
import java.util.concurrent.TimeUnit;
-import lombok.extern.slf4j.Slf4j;
+import lombok.CustomLog;
import org.apache.bookkeeper.common.util.MathUtils;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
-@Slf4j
+@CustomLog
class ListenerBasedPendingReadOp extends PendingReadOp {
final ReadEntryListener listener;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
index 83c2b4ada5f..855564304e4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/MetadataUpdateLoop.java
@@ -20,6 +20,7 @@
package org.apache.bookkeeper.client;
import com.google.common.util.concurrent.RateLimiter;
+import io.github.merlimat.slog.Logger;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.function.Supplier;
@@ -27,8 +28,6 @@
import org.apache.bookkeeper.meta.LedgerManager;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Mechanism to safely update the metadata of a ledger.
@@ -47,7 +46,8 @@
* iteration of the loop begins.
*/
class MetadataUpdateLoop {
- static final Logger LOG = LoggerFactory.getLogger(MetadataUpdateLoop.class);
+
+ private final Logger log;
private final LedgerManager lm;
private final long ledgerId;
@@ -57,7 +57,6 @@ class MetadataUpdateLoop {
private final LocalValueUpdater updateLocalValue;
private final RateLimiter throttler;
- private final String logContext;
private volatile int writeLoopCount = 0;
private static final AtomicIntegerFieldUpdater WRITE_LOOP_COUNT_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(MetadataUpdateLoop.class, "writeLoopCount");
@@ -111,7 +110,10 @@ interface LocalValueUpdater {
this.updateLocalValue = updateLocalValue;
this.throttler = throttler;
- this.logContext = String.format("UpdateLoop(ledgerId=%d,loopId=%08x)", ledgerId, System.identityHashCode(this));
+ this.log = Logger.get(MetadataUpdateLoop.class).with()
+ .attr("ledgerId", ledgerId)
+ .attr("loopId", String.format("%08x", System.identityHashCode(this)))
+ .build();
}
CompletableFuture> run() {
@@ -124,10 +126,11 @@ CompletableFuture> run() {
private void writeLoop(Versioned currentLocal,
CompletableFuture> promise) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} starting write loop iteration, attempt {}",
- logContext, WRITE_LOOP_COUNT_UPDATER.incrementAndGet(this));
- }
+
+ log.debug()
+ .attr("attempt", WRITE_LOOP_COUNT_UPDATER.incrementAndGet(this))
+ .log("Starting write loop iteration");
+
try {
if (needsTransformation.needsUpdate(currentLocal.getValue())) {
LedgerMetadata transformed = transform.transform(currentLocal.getValue());
@@ -139,20 +142,18 @@ private void writeLoop(Versioned currentLocal,
.whenComplete((writtenMetadata, ex) -> {
if (ex == null) {
if (updateLocalValue.updateValue(currentLocal, writtenMetadata)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} success", logContext);
- }
+
+ log.debug("success");
+
promise.complete(writtenMetadata);
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} local value changed while we were writing, try again",
- logContext);
- }
+
+ log.debug("local value changed while we were writing, try again");
+
writeLoop(currentLocalValue.get(), promise);
}
} else if (ex instanceof BKException.BKMetadataVersionException) {
- LOG.info("{} conflict writing metadata to store, update local value and try again",
- logContext);
+ log.info("conflict writing metadata to store, update local value and try again");
updateLocalValueFromStore(ledgerId).whenComplete((readMetadata, readEx) -> {
if (readEx == null) {
writeLoop(readMetadata, promise);
@@ -161,18 +162,22 @@ private void writeLoop(Versioned currentLocal,
}
});
} else {
- LOG.error("{} Error writing metadata to store", logContext, ex);
+ log.error()
+ .exception(ex)
+ .log("Error writing metadata to store");
promise.completeExceptionally(ex);
}
});
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} Update not needed, completing", logContext);
- }
+
+ log.debug("Update not needed, completing");
+
promise.complete(currentLocal);
}
} catch (Exception e) {
- LOG.error("{} Exception updating", logContext, e);
+ log.error()
+ .exception(e)
+ .log("Exception updating");
promise.completeExceptionally(e);
}
}
@@ -191,8 +196,9 @@ private void readLoop(long ledgerId, CompletableFuture
lm.readLedgerMetadata(ledgerId).whenComplete(
(read, exception) -> {
if (exception != null) {
- LOG.error("{} Failed to read metadata from store",
- logContext, exception);
+ log.error()
+ .exception(exception)
+ .log("Failed to read metadata from store");
promise.completeExceptionally(exception);
} else if (current.getVersion().compare(read.getVersion()) == Version.Occurred.CONCURRENTLY) {
// no update needed, these are the same in the immutable world
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
index 6b82200b3fa..cd55eb69ee5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
@@ -35,13 +35,12 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.AsyncCallback.AddCallbackWithLatency;
import org.apache.bookkeeper.client.api.WriteFlag;
import org.apache.bookkeeper.common.util.MathUtils;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This represents a pending add operation. When it has got success from all
@@ -52,8 +51,8 @@
*
*
*/
+@CustomLog
class PendingAddOp implements WriteCallback {
- private static final Logger LOG = LoggerFactory.getLogger(PendingAddOp.class);
ByteBuf payload;
ReferenceCounted toSend;
@@ -209,10 +208,13 @@ synchronized void unsetSuccessAndSendWriteRequest(List ensemble, int b
return;
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Unsetting success for ledger: " + lh.ledgerId + " entry: " + entryId + " bookie index: "
- + bookieIndex);
- }
+
+ log.debug()
+ .attr("ledgerId", lh.ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieIndex", bookieIndex)
+ .log("Unsetting success");
+
// if we had already heard a success from this array index, need to
// increment our number of responses that are pending, since we are
@@ -266,9 +268,12 @@ public synchronized void writeComplete(int rc, long ledgerId, long entryId, Book
if (!ensemble.get(bookieIndex).equals(addr)) {
// ensemble has already changed, failure of this addr is immaterial
- if (LOG.isDebugEnabled()) {
- LOG.debug("Write did not succeed: " + ledgerId + ", " + entryId + ". But we have already fixed it.");
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .log("Write did not succeed. But we have already fixed it.");
+
return;
}
@@ -324,13 +329,19 @@ public synchronized void writeComplete(int rc, long ledgerId, long entryId, Book
lh.handleUnrecoverableErrorDuringAdd(rc);
return;
case BKException.Code.LedgerFencedException:
- LOG.warn("Fencing exception on write: L{} E{} on {}",
- ledgerId, entryId, addr);
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .log("Fencing exception on write");
lh.handleUnrecoverableErrorDuringAdd(rc);
return;
case BKException.Code.UnauthorizedAccessException:
- LOG.warn("Unauthorized access exception on write: L{} E{} on {}",
- ledgerId, entryId, addr);
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", addr)
+ .log("Unauthorized access exception on write");
lh.handleUnrecoverableErrorDuringAdd(rc);
return;
default:
@@ -338,18 +349,31 @@ public synchronized void writeComplete(int rc, long ledgerId, long entryId, Book
if (ackSet.failBookieAndCheck(bookieIndex, addr)
|| rc == BKException.Code.WriteOnReadOnlyBookieException) {
Map failedBookies = ackSet.getFailedBookies();
- LOG.warn("Failed to write entry ({}, {}) to bookies {}, handling failures.",
- ledgerId, entryId, failedBookies);
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("failedBookies", failedBookies)
+ .log("Failed to write entry to bookies, handling failures");
// we can't meet ack quorum requirement, trigger ensemble change.
lh.handleBookieFailure(failedBookies);
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("Failed to write entry ({}, {}) to bookie ({}, {}),"
- + " but it didn't break ack quorum, delaying ensemble change : {}",
- ledgerId, entryId, bookieIndex, addr, BKException.getMessage(rc));
+ } else {
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieIndex", bookieIndex)
+ .attr("bookieAddr", addr)
+ .attr("error", BKException.getMessage(rc))
+ .log("Failed to write entry to bookie,"
+ + " but it didn't break ack quorum, delaying ensemble change");
}
} else {
- LOG.warn("Failed to write entry ({}, {}) to bookie ({}, {}): {}",
- ledgerId, entryId, bookieIndex, addr, BKException.getMessage(rc));
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieIndex", bookieIndex)
+ .attr("bookieAddr", addr)
+ .attr("error", BKException.getMessage(rc))
+ .log("Failed to write entry to bookie");
lh.handleBookieFailure(ImmutableMap.of(bookieIndex, addr));
}
return;
@@ -361,8 +385,10 @@ public synchronized void writeComplete(int rc, long ledgerId, long entryId, Book
.areAckedBookiesAdheringToPlacementPolicy(addEntrySuccessBookies,
lh.getLedgerMetadata().getWriteQuorumSize(),
lh.getLedgerMetadata().getAckQuorumSize()))) {
- LOG.warn("Write success for entry ID {} delayed, not acknowledged by bookies in enough fault domains",
- entryId);
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .log("Write success for entry ID delayed, not acknowledged by bookies in enough fault domains");
// Increment to indicate write did not complete due to not enough fault domains
clientCtx.getClientStats().getWriteDelayedDueToNotEnoughFaultDomains().inc();
@@ -391,15 +417,21 @@ void sendAddSuccessCallbacks() {
}
synchronized void submitCallback(final int rc) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Submit callback (lid:{}, eid: {}). rc:{}", lh.getId(), entryId, rc);
- }
+
+ log.debug()
+ .attr("ledgerId", lh.getId())
+ .attr("entryId", entryId)
+ .attr("rc", rc)
+ .log("Submit callback");
+
long latencyNanos = MathUtils.elapsedNanos(requestTimeNanos);
if (rc != BKException.Code.OK) {
clientCtx.getClientStats().getAddOpLogger().registerFailedEvent(latencyNanos, TimeUnit.NANOSECONDS);
- LOG.error("Write of ledger entry to quorum failed: L{} E{}",
- lh.getId(), entryId);
+ log.error()
+ .attr("ledgerId", lh.getId())
+ .attr("entryId", entryId)
+ .log("Write of ledger entry to quorum failed");
} else {
clientCtx.getClientStats().getAddOpLogger().registerSuccessfulEvent(latencyNanos, TimeUnit.NANOSECONDS);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java
index 07687bfcce0..896d3478ce6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadLacOp.java
@@ -19,13 +19,12 @@
import io.netty.buffer.ByteBuf;
import java.util.List;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadLacCallback;
import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This represents a pending ReadLac operation.
@@ -42,8 +41,8 @@
* from bookies, and doesn't affect the correctness of the protocol.
*/
+@CustomLog
class PendingReadLacOp implements ReadLacCallback {
- static final Logger LOG = LoggerFactory.getLogger(PendingReadLacOp.class);
LedgerHandle lh;
BookieClient bookieClient;
LacCallback cb;
@@ -121,8 +120,10 @@ public void readLacComplete(int rc, long ledgerId, final ByteBuf lacBuffer, fina
} catch (BKDigestMatchException e) {
// Too bad, this bookie did not give us a valid answer, we
// still might be able to recover. So, continue
- LOG.error("Mac mismatch while reading ledger: " + ledgerId + " LAC from bookie: "
- + currentEnsemble.get(bookieIndex));
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("bookieAddr", currentEnsemble.get(bookieIndex))
+ .log("Mac mismatch while reading LAC from bookie");
rc = BKException.Code.DigestMatchException;
}
}
@@ -146,17 +147,21 @@ public void readLacComplete(int rc, long ledgerId, final ByteBuf lacBuffer, fina
&& coverageSet.checkCovered()
&& !completed) {
completed = true;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Read LAC complete with enough validResponse for ledger: {} LAC: {}", ledgerId, maxLac);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("maxLac", maxLac)
+ .log("Read LAC complete with enough validResponse");
+
cb.getLacComplete(BKException.Code.OK, maxLac);
return;
}
if (numResponsesPending == 0 && !completed) {
- LOG.error(
- "While readLac ledger: {} did not hear success responses from all of ensemble, coverageSet is: {}",
- ledgerId, coverageSet);
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("coverageSet", coverageSet)
+ .log("While readLac did not hear success responses from all of ensemble");
cb.getLacComplete(lastSeenError, maxLac);
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
index b45617be55f..79cfa41bfa4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
@@ -26,6 +26,7 @@
import java.util.LinkedList;
import java.util.List;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
import org.apache.bookkeeper.common.util.MathUtils;
@@ -33,8 +34,6 @@
import org.apache.bookkeeper.proto.BookieProtocol;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
import org.apache.bookkeeper.proto.checksum.DigestManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Sequence of entries of a ledger that represents a pending read operation.
@@ -43,8 +42,8 @@
* application as soon as it arrives rather than waiting for the whole thing.
*
*/
+@CustomLog
class PendingReadOp extends ReadOpBase implements ReadEntryCallback {
- private static final Logger LOG = LoggerFactory.getLogger(PendingReadOp.class);
protected boolean parallelRead = false;
protected final LinkedList seq;
@@ -118,8 +117,11 @@ public void readEntryComplete(int rc, long ledgerId, final long entryId, final B
}
if (numPendingEntries < 0) {
- LOG.error("Read too many values for ledger {} : [{}, {}].",
- ledgerId, startEntryId, endEntryId);
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .log("Read too many values");
}
}
@@ -150,11 +152,17 @@ protected void submitCallback(int code) {
break;
}
}
- LOG.error(
- "Read of ledger entry failed: L{} E{}-E{}, Sent to {}, "
- + "Heard from {} : bitset = {}, Error = '{}'. First unread entry is ({}, rc = {})",
- lh.getId(), startEntryId, endEntryId, sentToHosts, heardFromHosts, heardFromHostsBitSet,
- BKException.getMessage(code), firstUnread, firstRc);
+ log.error()
+ .attr("ledgerId", lh.getId())
+ .attr("startEntryId", startEntryId)
+ .attr("endEntryId", endEntryId)
+ .attr("sentToHosts", sentToHosts)
+ .attr("heardFromHosts", heardFromHosts)
+ .attr("heardFromHostsBitSet", heardFromHostsBitSet)
+ .attr("error", BKException.getMessage(code))
+ .attr("firstUnreadEntry", firstUnread)
+ .attr("firstReturnCode", firstRc)
+ .log("Read of ledger entry failed");
clientCtx.getClientStats().getReadOpLogger().registerFailedEvent(latencyNanos, TimeUnit.NANOSECONDS);
// release the entries
seq.forEach(LedgerEntryRequest::close);
@@ -248,7 +256,10 @@ void read() {
try {
sendReadTo(writeSet.get(i), to, this);
} catch (InterruptedException ie) {
- LOG.error("Interrupted reading entry {} : ", this, ie);
+ log.error()
+ .exception(ie)
+ .attr("readOp", this)
+ .log("Interrupted reading entry");
Thread.currentThread().interrupt();
fail(BKException.Code.InterruptedException);
return;
@@ -363,7 +374,10 @@ synchronized BookieId sendNextRead() {
sentReplicas.set(replica);
return to;
} catch (InterruptedException ie) {
- LOG.error("Interrupted reading entry " + this, ie);
+ log.error()
+ .exception(ie)
+ .attr("readOp", this)
+ .log("Interrupted reading entry");
Thread.currentThread().interrupt();
fail(BKException.Code.InterruptedException);
return null;
@@ -376,7 +390,10 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, Strin
int replica = writeSet.indexOf(bookieIndex);
if (replica == NOT_FOUND) {
- LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble);
+ log.error()
+ .attr("bookieAddr", host)
+ .attr("ensemble", ensemble)
+ .log("Received error from a host which is not in the ensemble");
return;
}
erroredReplicas.set(replica);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java
index 32e5f8a848d..cc686e48619 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingWriteLacOp.java
@@ -20,12 +20,11 @@
import io.netty.util.ReferenceCountUtil;
import java.util.BitSet;
import java.util.List;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.AsyncCallback.AddLacCallback;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteLacCallback;
import org.apache.bookkeeper.util.ByteBufList;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This represents a pending WriteLac operation. When it has got
@@ -36,8 +35,8 @@
* to be up to date with the writer. This is best effort to get latest LAC
* from bookies, and doesn't affect the correctness of the protocol.
*/
+@CustomLog
class PendingWriteLacOp implements WriteLacCallback {
- private static final Logger LOG = LoggerFactory.getLogger(PendingWriteLacOp.class);
AddLacCallback cb;
long lac;
Object ctx;
@@ -110,7 +109,10 @@ public synchronized void writeLacComplete(int rc, long ledgerId, BookieId addr,
return;
}
} else {
- LOG.warn("WriteLac did not succeed: Ledger {} on {}", ledgerId, addr);
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("bookieAddr", addr)
+ .log("WriteLac did not succeed");
}
if (receivedResponseSet.isEmpty()){
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
index be90ceab94d..c96319cb30a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
@@ -46,6 +46,7 @@
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject;
@@ -70,8 +71,6 @@
import org.apache.bookkeeper.stats.annotations.StatsDoc;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Simple rackware ensemble placement policy.
@@ -82,9 +81,8 @@
name = CLIENT_SCOPE,
help = "BookKeeper client stats"
)
+@CustomLog
public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacementPolicy {
-
- static final Logger LOG = LoggerFactory.getLogger(RackawareEnsemblePlacementPolicyImpl.class);
int maxWeightMultiple;
protected int minNumRacksPerWriteQuorum;
@@ -248,23 +246,25 @@ public Integer getSample() {
? InetAddress.getLocalHost().getCanonicalHostName() : InetAddress.getLocalHost().getHostAddress();
bn = createDummyLocalBookieNode(hostname);
} catch (IOException e) {
- LOG.error("Failed to get local host address : ", e);
+ log.error().exception(e).log("Failed to get local host address");
}
} else {
- LOG.info("Ignoring LocalNode in Placementpolicy");
+ log.info("Ignoring LocalNode in Placementpolicy");
}
localNode = bn;
- LOG.info("Initialize rackaware ensemble placement policy @ {} @ {} : {}.",
- localNode, null == localNode ? "Unknown" : localNode.getNetworkLocation(),
- dnsResolver.getClass().getName());
+ log.info()
+ .attr("localNode", localNode)
+ .attr("getNetworkLocation", null == localNode ? "Unknown" : localNode.getNetworkLocation())
+ .attr("getName", dnsResolver.getClass().getName())
+ .log("Initialize rackaware ensemble placement policy");
this.isWeighted = isWeighted;
if (this.isWeighted) {
this.maxWeightMultiple = maxWeightMultiple;
this.weightedSelection = new WeightedRandomSelectionImpl(this.maxWeightMultiple);
- LOG.info("Weight based placement with max multiple of " + this.maxWeightMultiple);
+ log.info().attr("maxWeightMultiple", this.maxWeightMultiple).log("Weight based placement");
} else {
- LOG.info("Not weighted");
+ log.info("Not weighted");
}
return this;
}
@@ -311,8 +311,10 @@ public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration conf,
}
} catch (RuntimeException re) {
if (!conf.getEnforceMinNumRacksPerWriteQuorum()) {
- LOG.warn("Failed to initialize DNS Resolver {}, used default subnet resolver because {}",
- dnsResolverName, re.getMessage());
+ log.warn()
+ .attr("dnsResolverName", dnsResolverName)
+ .exceptionMessage(re)
+ .log("Failed to initialize DNS Resolver, used default subnet resolver");
dnsResolver = new DefaultResolver(this::getDefaultRack);
dnsResolver.setBookieAddressResolver(bookieAddressResolver);
} else {
@@ -370,7 +372,10 @@ protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced(
}
bookiesInDefaultRack.add(((BookieNode) node).getAddr());
} else {
- LOG.error("found non-BookieNode: {} as leaf of defaultrack: {}", node, getDefaultRack());
+ log.error()
+ .attr("node", node)
+ .attr("defaultRack", getDefaultRack())
+ .log("found non-BookieNode as leaf of defaultrack");
}
}
if ((bookiesInDefaultRack == null) || bookiesInDefaultRack.isEmpty()) {
@@ -378,8 +383,9 @@ protected Set addDefaultRackBookiesIfMinNumRacksIsEnforced(
} else {
comprehensiveExclusionBookiesSet = new HashSet(excludeBookies);
comprehensiveExclusionBookiesSet.addAll(bookiesInDefaultRack);
- LOG.info("enforceMinNumRacksPerWriteQuorum is enabled, so Excluding bookies of defaultRack: {}",
- bookiesInDefaultRack);
+ log.info()
+ .attr("bookiesInDefaultRack", bookiesInDefaultRack)
+ .log("enforceMinNumRacksPerWriteQuorum is enabled, so excluding bookies of defaultRack");
}
} else {
comprehensiveExclusionBookiesSet = excludeBookies;
@@ -445,7 +451,7 @@ protected PlacementResult> newEnsembleInternal(
// only one rack, use the random algorithm.
if (numRacks < 2) {
if (enforceMinNumRacksPerWriteQuorum && (minNumRacksPerWriteQuorumForThisEnsemble > 1)) {
- LOG.error("Only one rack available and minNumRacksPerWriteQuorum is enforced, so giving up");
+ log.error("Only one rack available and minNumRacksPerWriteQuorum is enforced, so giving up");
throw new BKNotEnoughBookiesException();
}
List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE,
@@ -487,8 +493,10 @@ protected PlacementResult> newEnsembleInternal(
}
List bookieList = ensemble.toList();
if (ensembleSize != bookieList.size()) {
- LOG.error("Not enough {} bookies are available to form an ensemble : {}.",
- ensembleSize, bookieList);
+ log.error()
+ .attr("ensembleSize", ensembleSize)
+ .attr("bookieList", bookieList)
+ .log("Not enough bookies are available to form an ensemble");
throw new BKNotEnoughBookiesException();
}
return PlacementResult.of(bookieList,
@@ -535,10 +543,13 @@ public PlacementResult replaceBookie(int ensembleSize, int writeQuorum
Set networkLocationsToBeExcluded = getNetworkLocations(ensembleNodes);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Try to choose a new bookie to replace {} from ensemble {}, excluding {}.",
- bookieToReplace, ensembleNodes, excludeNodes);
- }
+
+ log.debug()
+ .attr("bookieToReplace", bookieToReplace)
+ .attr("ensembleNodes", ensembleNodes)
+ .attr("excludeNodes", excludeNodes)
+ .log("Try to choose a new bookie to replace from ensemble");
+
// pick a candidate from same rack to replace
BookieNode candidate = selectFromNetworkLocation(
bn.getNetworkLocation(),
@@ -547,9 +558,12 @@ public PlacementResult replaceBookie(int ensembleSize, int writeQuorum
TruePredicate.INSTANCE,
EnsembleForReplacementWithNoConstraints.INSTANCE,
!enforceMinNumRacksPerWriteQuorum);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bn);
- }
+
+ log.debug()
+ .attr("candidate", candidate)
+ .attr("bookieNode", bn)
+ .log("Bookie is chosen to replace bookie");
+
BookieId candidateAddr = candidate.getAddr();
List newEnsemble = new ArrayList(currentEnsemble);
if (currentEnsemble.isEmpty()) {
@@ -581,16 +595,19 @@ public BookieNode selectFromNetworkLocation(
return selectRandomFromRack(networkLoc, excludeBookies, predicate, ensemble);
} catch (BKNotEnoughBookiesException e) {
if (!fallbackToRandom) {
- LOG.error(
- "Failed to choose a bookie from {} : "
- + "excluded {}, enforceMinNumRacksPerWriteQuorum is enabled so giving up.",
- networkLoc, excludeBookies);
+ log.error()
+ .attr("networkLoc", networkLoc)
+ .attr("excludeBookies", excludeBookies)
+ .log("Failed to choose a bookie, enforceMinNumRacksPerWriteQuorum is enabled so giving up");
throw e;
}
- LOG.warn("Failed to choose a bookie from network location {}, "
- + "the bookies in the network location are {}, excluded bookies {}, "
- + "current ensemble {}, fallback to choose bookie randomly from the cluster.",
- networkLoc, topology.getLeaves(networkLoc), excludeBookies, ensemble.toList());
+ log.warn()
+ .attr("networkLoc", networkLoc)
+ .attr("getLeaves", topology.getLeaves(networkLoc))
+ .attr("excludeBookies", excludeBookies)
+ .attr("toList", ensemble.toList())
+ .log("Failed to choose a bookie from network location, "
+ + "fallback to choose bookie randomly from the cluster");
// randomly choose one from whole cluster, ignore the provided predicate.
return selectRandom(1, excludeBookies, predicate, ensemble).get(0);
}
@@ -613,10 +630,13 @@ public BookieNode selectFromNetworkLocation(String networkLoc,
* the whole cluster and exclude the racks specified at
* excludeRacks.
*/
- LOG.warn("Failed to choose a bookie node from network location {}, "
- + "the bookies in the network location are {}, excluded bookies {}, "
- + "current ensemble {}, fallback to choose bookie randomly from the cluster.",
- networkLoc, topology.getLeaves(networkLoc), excludeBookies, ensemble.toList());
+ log.warn()
+ .attr("networkLoc", networkLoc)
+ .attr("getLeaves", topology.getLeaves(networkLoc))
+ .attr("excludeBookies", excludeBookies)
+ .attr("toList", ensemble.toList())
+ .log("Failed to choose a bookie node from network location, "
+ + "fallback to choose bookie randomly from the cluster");
return selectFromNetworkLocation(excludeRacks, excludeBookies, predicate, ensemble, fallbackToRandom);
}
}
@@ -647,15 +667,17 @@ public BookieNode selectFromNetworkLocation(Set excludeRacks,
return selectRandomInternal(knownNodes, 1, fullExclusionBookiesList, predicate, ensemble).get(0);
} catch (BKNotEnoughBookiesException e) {
if (!fallbackToRandom) {
- LOG.error(
- "Failed to choose a bookie excluding Racks: {} "
- + "Nodes: {}, enforceMinNumRacksPerWriteQuorum is enabled so giving up.",
- excludeRacks, excludeBookies);
+ log.error()
+ .attr("excludeRacks", excludeRacks)
+ .attr("excludeBookies", excludeBookies)
+ .log("Failed to choose a bookie excluding Racks: "
+ + "Nodes: enforceMinNumRacksPerWriteQuorum is enabled so giving up.");
throw e;
}
- LOG.warn("Failed to choose a bookie: excluded {}, fallback to choose bookie randomly from the cluster.",
- excludeBookies);
+ log.warn()
+ .attr("excludeBookies", excludeBookies)
+ .log("Failed to choose a bookie, fallback to choose bookie randomly from the cluster");
// randomly choose one from whole cluster
return selectRandom(1, excludeBookies, predicate, ensemble).get(0);
}
@@ -845,8 +867,11 @@ protected List selectRandomInternal(List bookiesToSelect
if (numBookies == 0) {
return newBookies;
}
- LOG.warn("Failed to find {} bookies : excludeBookies {}, allBookies {}.",
- numBookies, excludeBookies, bookiesToSelectFrom);
+ log.warn()
+ .attr("numBookies", numBookies)
+ .attr("excludeBookies", excludeBookies)
+ .attr("bookiesToSelectFrom", bookiesToSelectFrom)
+ .log("Failed to find bookies");
throw new BKNotEnoughBookiesException();
}
@@ -957,11 +982,14 @@ DistributionSchedule.WriteSet reorderReadSequenceWithRegion(
// to not trigger the speculativeReadTimeout. But even if it hits that timeout,
// things may have changed by then so much that whichever bookie we put second
// may actually not be the second-best choice any more.
- if (LOG.isDebugEnabled()) {
- LOG.debug("read set reordered from {} ({} pending) to {} ({} pending)",
- ensemble.get(writeSet.get(0)), pendingReqs[0], ensemble.get(writeSet.get(bestBookieIdx)),
- pendingReqs[bestBookieIdx]);
- }
+
+ log.debug()
+ .attr("fromBookie", ensemble.get(writeSet.get(0)))
+ .attr("fromPending", pendingReqs[0])
+ .attr("toBookie", ensemble.get(writeSet.get(bestBookieIdx)))
+ .attr("toPending", pendingReqs[bestBookieIdx])
+ .log("read set reordered");
+
writeSet.moveAndShift(bestBookieIdx, 0);
reordered = true;
}
@@ -1103,8 +1131,8 @@ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List ackedBooki
for (BookieId bookie : ackedBookies) {
if (knownBookies.containsKey(bookie)) {
rackCounter.add(knownBookies.get(bookie).getNetworkLocation());
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("bookie {} is not in the list of knownBookies", bookie);
+ } else {
+ log.debug().attr("bookieAddr", bookie).log("bookie is not in the list of knownBookies");
}
}
// Check to make sure that ensemble is writing to `minNumberOfRacks`'s number of racks at least.
- if (LOG.isDebugEnabled()) {
- LOG.debug("areAckedBookiesAdheringToPlacementPolicy returning {} because number of racks = {} and "
- + "minNumRacksPerWriteQuorum = {}",
- rackCounter.size() >= minNumRacksPerWriteQuorum,
- rackCounter.size(),
- minNumRacksPerWriteQuorum);
- }
+
+ log.debug()
+ .attr("result", rackCounter.size() >= minNumRacksPerWriteQuorum)
+ .attr("rackCount", rackCounter.size())
+ .attr("minNumRacksPerWriteQuorum", minNumRacksPerWriteQuorum)
+ .log("areAckedBookiesAdheringToPlacementPolicy returning because number of racks"
+ + " and minNumRacksPerWriteQuorum");
+
} finally {
readLock.unlock();
}
@@ -1178,7 +1210,9 @@ public PlacementResult> replaceToAdherePlacementPolicy(
int numRacks = topology.getNumOfRacks();
// only one rack or less than minNumRacksPerWriteQuorumForThisEnsemble, stop calculation to skip relocation
if (numRacks < 2 || numRacks < minNumRacksPerWriteQuorumForThisEnsemble) {
- LOG.warn("Skip ensemble relocation because the cluster has only {} rack.", numRacks);
+ log.warn()
+ .attr("numRacks", numRacks)
+ .log("Skip ensemble relocation because the cluster has only rack.");
return PlacementResult.of(Collections.emptyList(), PlacementPolicyAdherence.FAIL);
}
PlacementResult> placementResult = PlacementResult.of(Collections.emptyList(),
@@ -1260,14 +1294,16 @@ private PlacementResult> doReplaceToAdherePlacementPolicy(
// replace to newer node
provisionalEnsembleNodes.set(index, prevNode);
} catch (BKNotEnoughBookiesException e) {
- LOG.warn("Skip ensemble relocation because the cluster has not enough bookies.");
+ log.warn("Skip ensemble relocation because the cluster has not enough bookies.");
return PlacementResult.of(Collections.emptyList(), PlacementPolicyAdherence.FAIL);
}
}
List bookieList = ensemble.toList();
if (ensembleSize != bookieList.size()) {
- LOG.warn("Not enough {} bookies are available to form an ensemble : {}.",
- ensembleSize, bookieList);
+ log.warn()
+ .attr("ensembleSize", ensembleSize)
+ .attr("bookieList", bookieList)
+ .log("Not enough bookies are available to form an ensemble");
return PlacementResult.of(Collections.emptyList(), PlacementPolicyAdherence.FAIL);
}
PlacementPolicyAdherence placementPolicyAdherence = isEnsembleAdheringToPlacementPolicy(bookieList,
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
index 35ba2823365..e187192d8c1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedAndEntryOp.java
@@ -21,6 +21,7 @@
package org.apache.bookkeeper.client;
import com.google.common.util.concurrent.ListenableFuture;
+import io.github.merlimat.slog.Logger;
import io.netty.buffer.ByteBuf;
import java.util.BitSet;
import java.util.List;
@@ -36,8 +37,6 @@
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
import org.apache.bookkeeper.proto.ReadLastConfirmedAndEntryContext;
import org.apache.bookkeeper.proto.checksum.DigestManager;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Long poll read operation.
@@ -45,8 +44,6 @@
class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEntryCallback,
SpeculativeRequestExecutor {
- static final Logger LOG = LoggerFactory.getLogger(ReadLastConfirmedAndEntryOp.class);
-
ReadLACAndEntryRequest request;
final BitSet heardFromHostsBitSet;
final BitSet emptyResponsesFromHostsBitSet;
@@ -67,6 +64,7 @@ class ReadLastConfirmedAndEntryOp implements BookkeeperInternalCallbacks.ReadEnt
private long timeOutInMillis;
private final List currentEnsemble;
private ScheduledFuture> speculativeTask = null;
+ protected final Logger log;
abstract class ReadLACAndEntryRequest implements AutoCloseable {
@@ -203,10 +201,13 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, Strin
++numMissedEntryReads;
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("{} while reading entry: {} ledgerId: {} from bookie: {}", errMsg, entryImpl.getEntryId(),
- lh.getId(), host);
- }
+
+ log.debug()
+ .attr("error", errMsg)
+ .attr("entryId", entryImpl.getEntryId())
+ .attr("bookieAddr", host)
+ .log("Error while reading entry from bookie");
+
}
/**
@@ -259,7 +260,9 @@ void read() {
try {
sendReadTo(orderedEnsemble.get(i), to, this);
} catch (InterruptedException ie) {
- LOG.error("Interrupted reading entry {} : ", this, ie);
+ log.error()
+ .exception(ie)
+ .log("Interrupted reading entry");
Thread.currentThread().interrupt();
fail(BKException.Code.InterruptedException);
return;
@@ -383,7 +386,10 @@ synchronized BookieId sendNextRead() {
sentReplicas.set(replica);
return to;
} catch (InterruptedException ie) {
- LOG.error("Interrupted reading entry " + this, ie);
+ log.error()
+ .attr("addEntryOp", this)
+ .exception(ie)
+ .log("Interrupted reading entry");
Thread.currentThread().interrupt();
fail(BKException.Code.InterruptedException);
return null;
@@ -396,7 +402,10 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, Strin
int replica = getReplicaIndex(bookieIndex);
if (replica == NOT_FOUND) {
- LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble);
+ log.error()
+ .attr("bookieAddr", host)
+ .attr("ensemble", ensemble)
+ .log("Received error from a host which is not in the ensemble");
return;
}
@@ -454,6 +463,11 @@ boolean complete(int bookieIndex, BookieId host, ByteBuf buffer, long entryId) {
- getLedgerMetadata().getAckQuorumSize();
heardFromHostsBitSet = new BitSet(getLedgerMetadata().getEnsembleSize());
emptyResponsesFromHostsBitSet = new BitSet(getLedgerMetadata().getEnsembleSize());
+ this.log = Logger.get(ReadLastConfirmedAndEntryOp.class)
+ .with()
+ .attr("ledgerId", lh.getId())
+ .attr("prevEntryId", prevEntryId)
+ .build();
}
protected LedgerMetadata getLedgerMetadata() {
@@ -481,10 +495,12 @@ public ListenableFuture issueSpeculativeRequest() {
public Boolean call() throws Exception {
if (!requestComplete.get() && !request.isComplete()
&& (null != request.maybeSendSpeculativeRead(heardFromHostsBitSet))) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Send speculative ReadLAC {} for ledger {} (previousLAC: {}). Hosts heard are {}.",
- request, lh.getId(), lastAddConfirmed, heardFromHostsBitSet);
- }
+
+ log.debug()
+ .attr("request", request)
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .attr("heardFromHostsBitSet", heardFromHostsBitSet)
+ .log("Send speculative ReadLAC for ledger");
return true;
}
return false;
@@ -507,10 +523,12 @@ public void initiate() {
}
void sendReadTo(int bookieIndex, BookieId to, ReadLACAndEntryRequest entry) throws InterruptedException {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Calling Read LAC and Entry with {} and long polling interval {} on Bookie {} - Parallel {}",
- prevEntryId, timeOutInMillis, to, parallelRead);
- }
+ log.debug()
+ .attr("timeOutInMillis", timeOutInMillis)
+ .attr("bookieAddr", to)
+ .attr("parallelRead", parallelRead)
+ .log("Calling Read LAC and Entry with and long polling interval on Bookie - Parallel");
+
clientCtx.getBookieClient().readEntryWaitForLACUpdate(to,
lh.getId(),
BookieProtocol.LAST_ADD_CONFIRMED,
@@ -552,18 +570,20 @@ private void submitCallback(int rc) {
@Override
public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("{} received response for (lid={}, eid={}) : {}",
- getClass().getName(), ledgerId, entryId, rc);
- }
+ log.trace()
+ .attr("class", getClass().getName())
+ .attr("entryId", entryId)
+ .attr("rc", rc)
+ .log("received response");
+
ReadLastConfirmedAndEntryContext rCtx = (ReadLastConfirmedAndEntryContext) ctx;
BookieId bookie = rCtx.getBookieAddress();
numResponsesPending--;
if (BKException.Code.OK == rc) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("Received lastAddConfirmed (lac={}) from bookie({}) for (lid={}).",
- rCtx.getLastAddConfirmed(), bookie, ledgerId);
- }
+ log.trace()
+ .attr("getLastAddConfirmed", rCtx.getLastAddConfirmed())
+ .attr("bookieAddr", bookie)
+ .log("Received lastAddConfirmed");
if (rCtx.getLastAddConfirmed() > lastAddConfirmed) {
lastAddConfirmed = rCtx.getLastAddConfirmed();
@@ -598,20 +618,19 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffe
// received advanced lac
completeRequest();
} else if (emptyResponsesFromHostsBitSet.cardinality() >= numEmptyResponsesAllowed) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Completed readLACAndEntry(lid = {}, previousEntryId = {}) "
- + "after received {} empty responses ('{}').",
- ledgerId, prevEntryId, emptyResponsesFromHostsBitSet.cardinality(),
- emptyResponsesFromHostsBitSet);
- }
+ log.debug()
+ .attr("cardinality", emptyResponsesFromHostsBitSet.cardinality())
+ .attr("emptyResponsesFromHostsBitSet", emptyResponsesFromHostsBitSet)
+ .log("Completed readLACAndEntry after received empty responses");
+
completeRequest();
} else {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Received empty response for readLACAndEntry(lid = {}, previousEntryId = {}) from"
- + " bookie {} @ {}, reattempting reading next bookie : lac = {}",
- ledgerId, prevEntryId, rCtx.getBookieAddress(),
- rCtx.getBookieAddress(), lastAddConfirmed);
- }
+ log.debug()
+ .attr("bookieAddr", rCtx.getBookieAddress())
+ .attr("bookieAddr", rCtx.getBookieAddress())
+ .attr("lastAddConfirmed", lastAddConfirmed)
+ .log("Received empty response from bookie, reattempting reading next bookie");
+
request.logErrorAndReattemptRead(rCtx.getBookieIndex(), bookie, "Empty Response", rc);
}
return;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
index 4e30c7231eb..83c3fcc62ff 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
@@ -20,6 +20,7 @@
import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
import java.util.List;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.BKDigestMatchException;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
@@ -27,15 +28,13 @@
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
import org.apache.bookkeeper.proto.checksum.DigestManager;
import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This class encapsulated the read last confirmed operation.
*
*/
+@CustomLog
class ReadLastConfirmedOp implements ReadEntryCallback {
- static final Logger LOG = LoggerFactory.getLogger(ReadLastConfirmedOp.class);
private final long ledgerId;
private final byte[] ledgerKey;
private final BookieClient bookieClient;
@@ -113,9 +112,11 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi
} catch (BKDigestMatchException e) {
// Too bad, this bookie didn't give us a valid answer, we
// still might be able to recover though so continue
- LOG.error("Mac mismatch for ledger: " + ledgerId + ", entry: " + entryId
- + " while reading last entry from bookie: "
- + currentEnsemble.get(bookieIndex));
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", currentEnsemble.get(bookieIndex))
+ .log("Mac mismatch while reading last entry from bookie");
}
}
@@ -138,18 +139,22 @@ public synchronized void readEntryComplete(final int rc, final long ledgerId, fi
&& coverageSet.checkCovered()
&& !completed) {
completed = true;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Read Complete with enough validResponses for ledger: {}, entry: {}",
- ledgerId, entryId);
- }
+
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .log("Read complete with enough valid responses");
+
cb.readLastConfirmedDataComplete(BKException.Code.OK, maxRecoveredData);
return;
}
if (numResponsesPending == 0 && !completed) {
- LOG.error("While readLastConfirmed ledger: {} did not hear success responses from all quorums, {}",
- ledgerId, coverageSet);
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("coverageSet", coverageSet)
+ .log("While readLastConfirmed did not hear success responses from all quorums");
cb.readLastConfirmedDataComplete(lastSeenError, maxRecoveredData);
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
index ffad02d8f7c..e6e1db2b3b9 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
@@ -44,9 +44,6 @@
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.TimedGenericCallback;
import org.apache.bookkeeper.versioning.Version;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
/**
* Read only ledger handle. This ledger handle allows you to
* read from a ledger but not to write to it. It overrides all
@@ -54,7 +51,6 @@
* It should be returned for BookKeeper#openLedger operations.
*/
class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListener {
- private static final Logger LOG = LoggerFactory.getLogger(ReadOnlyLedgerHandle.class);
private Object metadataLock = new Object();
private final NavigableMap> newEnsemblesFromRecovery = new TreeMap<>();
@@ -75,8 +71,10 @@ public void run() {
if (Version.Occurred.BEFORE == occurred) {
synchronized (ReadOnlyLedgerHandle.this) {
if (setLedgerMetadata(currentMetadata, newMetadata)) {
- LOG.info("Updated ledger metadata for ledger {} to {}, version {}.",
- ledgerId, newMetadata.getValue().toSafeString(), newMetadata.getVersion());
+ log.info()
+ .attr("newMetadata", newMetadata.getValue().toSafeString())
+ .attr("newMetadataVersion", newMetadata.getVersion())
+ .log("Updated ledger metadata");
break;
}
}
@@ -127,7 +125,7 @@ public long addEntry(byte[] data) throws InterruptedException, BKException {
@Override
public long addEntry(byte[] data, int offset, int length)
throws InterruptedException, BKException {
- LOG.error("Tried to add entry on a Read-Only ledger handle, ledgerid=" + ledgerId);
+ log.error("Tried to add entry on a Read-Only ledger handle");
throw BKException.create(BKException.Code.IllegalOpException);
}
@@ -140,16 +138,18 @@ public void asyncAddEntry(final byte[] data, final AddCallback cb,
@Override
public void asyncAddEntry(final byte[] data, final int offset, final int length,
final AddCallback cb, final Object ctx) {
- LOG.error("Tried to add entry on a Read-Only ledger handle, ledgerid=" + ledgerId);
+ log.error("Tried to add entry on a Read-Only ledger handle");
cb.addComplete(BKException.Code.IllegalOpException, this,
LedgerHandle.INVALID_ENTRY_ID, ctx);
}
@Override
public void onChanged(long lid, Versioned newMetadata) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Received ledger metadata update on {} : {}", lid, newMetadata);
- }
+
+ log.debug()
+ .attr("newMetadata", newMetadata)
+ .log("Received ledger metadata update");
+
if (this.ledgerId != lid) {
return;
}
@@ -158,16 +158,20 @@ public void onChanged(long lid, Versioned newMetadata) {
}
Versioned currentMetadata = getVersionedLedgerMetadata();
Version.Occurred occurred = currentMetadata.getVersion().compare(newMetadata.getVersion());
- if (LOG.isDebugEnabled()) {
- LOG.debug("Try to update metadata from {} to {} : {}",
- currentMetadata, newMetadata, occurred);
- }
+
+ log.debug()
+ .attr("currentMetadata", currentMetadata)
+ .attr("newMetadata", newMetadata)
+ .attr("occurred", occurred)
+ .log("Try to update metadata");
+
if (Version.Occurred.BEFORE == occurred) { // the metadata is updated
try {
clientCtx.getMainWorkerPool().executeOrdered(ledgerId, new MetadataUpdater(newMetadata));
} catch (RejectedExecutionException ree) {
- LOG.error("Failed on submitting updater to update ledger metadata on ledger {} : {}",
- ledgerId, newMetadata);
+ log.error()
+ .attr("newMetadata", newMetadata)
+ .log("Failed on submitting updater to update ledger metadata on ledger");
}
}
}
@@ -196,8 +200,10 @@ public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
asyncReadEntriesInternal(lastConfirmed, lastConfirmed, cb, ctx, false);
}
} else {
- LOG.error("ReadException in asyncReadLastEntry, ledgerId: {}, lac: {}, rc:{}",
- lastConfirmed, ledgerId, rc);
+ log.error()
+ .attr("lastAddConfirmed", lastConfirmed)
+ .attr("rc", rc)
+ .log("ReadException in asyncReadLastEntry");
cb.readComplete(rc, ReadOnlyLedgerHandle.this, null, ctx);
}
}
@@ -217,21 +223,19 @@ void handleBookieFailure(final Map failedBookies) {
// ledger, so this synchronized should be unnecessary, but putting it here now
// just in case (can be removed when we validate threads)
synchronized (metadataLock) {
- String logContext = String.format("[RecoveryEnsembleChange(ledger:%d)]", ledgerId);
-
long lac = getLastAddConfirmed();
LedgerMetadata metadata = getLedgerMetadata();
List currentEnsemble = getCurrentEnsemble();
try {
List newEnsemble = EnsembleUtils.replaceBookiesInEnsemble(
- clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, logContext);
+ clientCtx.getBookieWatcher(), metadata, currentEnsemble, failedBookies, 0);
Set replaced = EnsembleUtils.diffEnsemble(currentEnsemble, newEnsemble);
if (!replaced.isEmpty()) {
newEnsemblesFromRecovery.put(lac + 1, newEnsemble);
unsetSuccessAndSendWriteRequest(newEnsemble, replaced);
}
} catch (BKException.BKNotEnoughBookiesException e) {
- LOG.error("Could not get additional bookie to remake ensemble, closing ledger: {}", ledgerId);
+ log.error("Could not get additional bookie to remake ensemble, closing ledger");
handleUnrecoverableErrorDuringAdd(e.getCode());
return;
@@ -306,7 +310,9 @@ CompletableFuture> closeRecovered() {
lac = lastAddConfirmed;
len = length.get();
}
- LOG.info("Closing recovered ledger {} at entry {}", getId(), lac);
+ log.info()
+ .attr("lastAddConfirmed", lac)
+ .log("Closing recovered ledger");
CompletableFuture> f = new MetadataUpdateLoop(
clientCtx.getLedgerManager(), getId(),
this::getVersionedLedgerMetadata,
@@ -335,7 +341,9 @@ CompletableFuture> closeRecovered() {
newEnsemblesFromRecovery.clear();
}
if (exception != null) {
- LOG.error("When closeRecovered,failed on clearing newEnsemblesFromRecovery.", exception);
+ log.error()
+ .exception(exception)
+ .log("When closeRecovered,failed on clearing newEnsemblesFromRecovery.");
}
});
return f;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOpBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOpBase.java
index cbd68ec657a..eb16f473c70 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOpBase.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOpBase.java
@@ -29,17 +29,15 @@
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.api.LedgerEntries;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+@CustomLog
public abstract class ReadOpBase implements Runnable {
- private static final Logger LOG = LoggerFactory.getLogger(ReadOpBase.class);
-
protected ScheduledFuture> speculativeTask = null;
protected final CompletableFuture future;
protected final Set heardFromHosts;
@@ -196,15 +194,22 @@ synchronized void logErrorAndReattemptRead(int bookieIndex, BookieId host, Strin
if (BKException.Code.NoSuchEntryException == rc
|| BKException.Code.NoSuchLedgerExistsException == rc) {
++numBookiesMissingEntry;
- if (LOG.isDebugEnabled()) {
- LOG.debug("No such entry found on bookie. L{} E{} bookie: {}",
- lh.ledgerId, eId, host);
- }
+
+ log.debug()
+ .attr("ledgerId", lh.ledgerId)
+ .attr("entryId", eId)
+ .attr("bookieAddr", host)
+ .log("No such entry found on bookie");
+
} else {
- if (LOG.isInfoEnabled()) {
- LOG.info("{} while reading L{} E{} from bookie: {}",
- errMsg, lh.ledgerId, eId, host);
- }
+
+ log.info()
+ .attr("error", errMsg)
+ .attr("ledgerId", lh.ledgerId)
+ .attr("entryId", eId)
+ .attr("bookieAddr", host)
+ .log("Error while reading from bookie");
+
}
lh.recordReadErrorOnBookie(bookieIndex);
@@ -255,11 +260,14 @@ public ListenableFuture issueSpeculativeRequest() {
@Override
public Boolean call() throws Exception {
if (!isComplete() && null != maybeSendSpeculativeRead(heardFromHostsBitSet)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Send speculative read for {}. Hosts sent are {}, "
- + " Hosts heard are {}, ensemble is {}.",
- this, sentToHosts, heardFromHostsBitSet, ensemble);
- }
+
+ log.debug()
+ .attr("readOp", this)
+ .attr("sentToHosts", sentToHosts)
+ .attr("heardFromHostsBitSet", heardFromHostsBitSet)
+ .attr("ensemble", ensemble)
+ .log("Send speculative read");
+
return true;
}
return false;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
index fd0bc8f7e5e..c6ae45d12a0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
@@ -28,6 +28,7 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import lombok.CustomLog;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.bookkeeper.feature.Feature;
import org.apache.bookkeeper.feature.FeatureProvider;
@@ -42,16 +43,14 @@
import org.apache.bookkeeper.stats.StatsLogger;
import org.apache.bookkeeper.util.BookKeeperConstants;
import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* A placement policy use region information in the network topology for placing ensembles.
*
* @see EnsemblePlacementPolicy
*/
+@CustomLog
public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlacementPolicy {
- static final Logger LOG = LoggerFactory.getLogger(RegionAwareEnsemblePlacementPolicy.class);
public static final String REPP_REGIONS_TO_WRITE = "reppRegionsToWrite";
public static final String REPP_MINIMUM_REGIONS_FOR_DURABILITY = "reppMinimumRegionsForDurability";
@@ -171,9 +170,9 @@ public void handleBookiesThatJoined(Set joinedBookies) {
regionSet.add(addr);
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : bookie {} joined the cluster.", addr);
- }
+
+ log.debug().attr("bookieAddr", addr).log("Cluster changed. New bookie joined the cluster.");
+
}
for (Map.Entry regionEntry : perRegionPlacement.entrySet()) {
@@ -229,7 +228,10 @@ public void onBookieRackChange(List bookieAddressList) {
}
}
} catch (IllegalArgumentException | NetworkTopologyImpl.InvalidTopologyException e) {
- LOG.error("Failed to update bookie rack info: {} ", bookieAddress, e);
+ log.error()
+ .exception(e)
+ .attr("bookieAddress", bookieAddress)
+ .log("Failed to update bookie rack info");
}
});
} finally {
@@ -356,7 +358,7 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
if (numRegionsAvailable < 1) {
// We cant disallow all regions; if we did, raise an alert to draw attention
if (perRegionPlacement.keySet().size() >= 1) {
- LOG.error("No regions available, invalid configuration");
+ log.error("No regions available, invalid configuration");
}
List bns = selectRandom(ensembleSize, excludeNodes, TruePredicate.INSTANCE,
EnsembleForReplacementWithNoConstraints.INSTANCE);
@@ -427,7 +429,7 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
TopologyAwareEnsemblePlacementPolicy policyWithinRegion = perRegionPlacement.get(region);
if (!regionsReachedMaxAllocation.contains(region)) {
if (numRemainingRegions <= 0) {
- LOG.error("Inconsistent State: This should never happen");
+ log.error("Inconsistent State: This should never happen");
throw new BKException.BKNotEnoughBookiesException();
}
// try to place the bookies as balance as possible across all the regions
@@ -456,13 +458,19 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
success = true;
regionsToAllocate--;
lastRegionIndex = startRegionIndex;
- LOG.info("Region {} allocating bookies with ensemble size {} "
- + "and write quorum size {} : {}",
- region, newEnsembleSize, newWriteQuorumSize, allocated);
+ log.info()
+ .attr("region", region)
+ .attr("ensembleSize", newEnsembleSize)
+ .attr("writeQuorumSize", newWriteQuorumSize)
+ .attr("allocated", allocated)
+ .log("Region allocating bookies");
break;
} catch (BKException.BKNotEnoughBookiesException exc) {
- LOG.warn("Could not allocate {} bookies in region {}, try allocating {} bookies",
- newEnsembleSize, region, (newEnsembleSize - 1));
+ log.warn()
+ .attr("ensembleSize", newEnsembleSize)
+ .attr("region", region)
+ .attr("suggestedEnsembleSize", (newEnsembleSize - 1))
+ .log("Could not allocate bookies in region, try allocating bookies");
addToEnsembleSize--;
}
}
@@ -476,8 +484,12 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
if (regionsReachedMaxAllocation.contains(region)) {
if (currentAllocation.getLeft() > 0) {
- LOG.info("Allocating {} bookies in region {} : ensemble {} exclude {}",
- currentAllocation.getLeft(), region, comprehensiveExclusionBookiesSet, ensemble);
+ log.info()
+ .attr("bookieCount", currentAllocation.getLeft())
+ .attr("region", region)
+ .attr("comprehensiveExclusionBookiesSet", comprehensiveExclusionBookiesSet)
+ .attr("ensemble", ensemble)
+ .log("Allocating bookies");
policyWithinRegion.newEnsemble(
currentAllocation.getLeft(),
currentAllocation.getRight(),
@@ -485,8 +497,11 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
comprehensiveExclusionBookiesSet,
ensemble,
ensemble);
- LOG.info("Allocated {} bookies in region {} : {}",
- currentAllocation.getLeft(), region, ensemble);
+ log.info()
+ .attr("bookiesCount", currentAllocation.getLeft())
+ .attr("region", region)
+ .attr("ensemble", ensemble)
+ .log("Allocated bookies");
}
}
}
@@ -498,17 +513,21 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
List bookieList = ensemble.toList();
if (ensembleSize != bookieList.size()) {
- LOG.error("Not enough {} bookies are available to form an ensemble : {}.",
- ensembleSize, bookieList);
+ log.error()
+ .attr("ensembleSize", ensembleSize)
+ .attr("bookieList", bookieList)
+ .log("Not enough bookies are available to form an ensemble");
throw new BKException.BKNotEnoughBookiesException();
}
if (enableValidation && !ensemble.validate()) {
- LOG.error("Not enough {} bookies are available to form a valid ensemble : {}.",
- ensembleSize, bookieList);
+ log.error()
+ .attr("ensembleSize", ensembleSize)
+ .attr("bookieList", bookieList)
+ .log("Not enough bookies are available to form a valid ensemble");
throw new BKException.BKNotEnoughBookiesException();
}
- LOG.info("Bookies allocated successfully {}", ensemble);
+ log.info().attr("ensemble", ensemble).log("Bookies allocated successfully");
List ensembleList = ensemble.toList();
return PlacementResult.of(ensembleList,
isEnsembleAdheringToPlacementPolicy(ensembleList, writeQuorumSize, ackQuorumSize));
@@ -555,7 +574,7 @@ public PlacementResult replaceBookie(int ensembleSize, int writeQuorum
excludeNodes.add(bn);
if (!ensemble.apply(bn, ensemble)) {
- LOG.warn("Anomalous ensemble detected");
+ log.warn("Anomalous ensemble detected");
if (null != statsLogger) {
statsLogger.getCounter(REGION_AWARE_ANOMALOUS_ENSEMBLE).inc();
}
@@ -565,16 +584,21 @@ public PlacementResult replaceBookie(int ensembleSize, int writeQuorum
ensemble.addNode(bn);
}
- if (LOG.isDebugEnabled()) {
- LOG.debug("Try to choose a new bookie to replace {}, excluding {}.", bookieToReplace,
- excludeNodes);
- }
+
+ log.debug()
+ .attr("bookieToReplace", bookieToReplace)
+ .attr("excludeNodes", excludeNodes)
+ .log("Trying to choose a new bookie to replace existing one");
+
// pick a candidate from same rack to replace
BookieNode candidate = replaceFromRack(bookieNodeToReplace, excludeNodes,
ensemble, ensemble, enforceDurability);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Bookie {} is chosen to replace bookie {}.", candidate, bookieNodeToReplace);
- }
+
+ log.debug()
+ .attr("candidate", candidate)
+ .attr("bookieNodeToReplace", bookieNodeToReplace)
+ .log("Bookie chosen to replace existing bookie");
+
BookieId candidateAddr = candidate.getAddr();
List newEnsemble = new ArrayList(currentEnsemble);
if (currentEnsemble.isEmpty()) {
@@ -623,9 +647,10 @@ protected BookieNode replaceFromRack(BookieNode bookieNodeToReplace,
EnsembleForReplacementWithNoConstraints.INSTANCE,
true);
} catch (BKException.BKNotEnoughBookiesException e) {
- LOG.warn("Failed to choose a bookie from {} : "
- + "excluded {}, fallback to choose bookie randomly from the cluster.",
- bookieNodeToReplace.getNetworkLocation(), excludeBookies);
+ log.warn()
+ .attr("getNetworkLocation", bookieNodeToReplace.getNetworkLocation())
+ .attr("excludeBookies", excludeBookies)
+ .log("Failed to choose a bookie, fallback to choose bookie randomly from the cluster");
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
index 62690129744..97319bc1dc7 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RoundRobinDistributionSchedule.java
@@ -25,10 +25,9 @@
import java.util.Arrays;
import java.util.BitSet;
import java.util.Map;
+import lombok.CustomLog;
import lombok.Getter;
import org.apache.bookkeeper.net.BookieId;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* A specific {@link DistributionSchedule} that places entries in round-robin
@@ -37,8 +36,8 @@
* on.
*
*/
+@CustomLog
public class RoundRobinDistributionSchedule implements DistributionSchedule {
- private static final Logger LOG = LoggerFactory.getLogger(RoundRobinDistributionSchedule.class);
@Getter
private final int writeQuorumSize;
private final int ackQuorumSize;
@@ -438,10 +437,12 @@ public boolean hasEntry(long entryId, int bookieIndex) {
public BitSet getEntriesStripedToTheBookie(int bookieIndex, long startEntryId, long lastEntryId) {
if ((startEntryId < 0) || (lastEntryId < 0) || (bookieIndex < 0) || (bookieIndex >= ensembleSize)
|| (lastEntryId < startEntryId)) {
- LOG.error(
- "Illegal arguments for getEntriesStripedToTheBookie, bookieIndex : {},"
- + " ensembleSize : {}, startEntryId : {}, lastEntryId : {}",
- bookieIndex, ensembleSize, startEntryId, lastEntryId);
+ log.error()
+ .attr("bookieIndex", bookieIndex)
+ .attr("ensembleSize", ensembleSize)
+ .attr("startEntryId", startEntryId)
+ .attr("entryId", lastEntryId)
+ .log("Illegal arguments for getEntriesStripedToTheBookie");
throw new IllegalArgumentException("Illegal arguments for getEntriesStripedToTheBookie");
}
BitSet entriesStripedToTheBookie = new BitSet((int) (lastEntryId - startEntryId + 1));
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SyncCallbackUtils.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SyncCallbackUtils.java
index a421a7547d7..8503739c1b4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SyncCallbackUtils.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/SyncCallbackUtils.java
@@ -22,7 +22,7 @@
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import lombok.extern.slf4j.Slf4j;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
import org.apache.bookkeeper.client.impl.LastConfirmedAndEntryImpl;
@@ -30,7 +30,7 @@
* Utility for callbacks.
*
*/
-@Slf4j
+@CustomLog
class SyncCallbackUtils {
/**
@@ -196,9 +196,9 @@ static class LastAddConfirmedCallback implements AsyncCallback.AddLacCallback {
@Override
public void addLacComplete(int rc, LedgerHandle lh, Object ctx) {
if (rc != BKException.Code.OK) {
- log.warn("LastAddConfirmedUpdate failed: {} ", BKException.getMessage(rc));
- } else if (log.isDebugEnabled()) {
- log.debug("Callback LAC Updated for: {} ", lh.getId());
+ log.warn().attr("error", BKException.getMessage(rc)).log("LastAddConfirmedUpdate failed");
+ } else {
+ log.debug().attr("ledgerId", lh.getId()).log("Callback LAC Updated");
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
index 9eae79438fd..be7dc952fc4 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
@@ -37,6 +37,7 @@
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.function.Supplier;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
import org.apache.bookkeeper.client.WeightedRandomSelection.WeightedObject;
import org.apache.bookkeeper.net.BookieId;
@@ -53,12 +54,10 @@
import org.apache.bookkeeper.stats.OpStatsLogger;
import org.apache.bookkeeper.stats.annotations.StatsDoc;
import org.apache.commons.collections4.CollectionUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+@CustomLog
abstract class TopologyAwareEnsemblePlacementPolicy implements
ITopologyAwareEnsemblePlacementPolicy {
- static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class);
public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass";
protected final Map knownBookies = new HashMap();
protected final Map historyBookies = new HashMap();
@@ -214,12 +213,14 @@ private boolean checkSumOfSubsetWithinLimit(final Set includedRacksOrReg
int maxAllowedSum) {
if (remainingRacksOrRegions.isEmpty() || (subsetSize <= 0)) {
if (maxAllowedSum < 0) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(
- "CHECK FAILED: RacksOrRegions Included {} Remaining {}, subsetSize {}, "
- + "maxAllowedSum {}",
- includedRacksOrRegions, remainingRacksOrRegions, subsetSize, maxAllowedSum);
- }
+
+ log.trace()
+ .attr("includedRacksOrRegions", includedRacksOrRegions)
+ .attr("remainingRacksOrRegions", remainingRacksOrRegions)
+ .attr("subsetSize", subsetSize)
+ .attr("maxAllowedSum", maxAllowedSum)
+ .log("CHECK FAILED: RacksOrRegions");
+
}
return (maxAllowedSum >= 0);
}
@@ -232,12 +233,14 @@ private boolean checkSumOfSubsetWithinLimit(final Set includedRacksOrReg
}
if (currentAllocation > maxAllowedSum) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(
- "CHECK FAILED: RacksOrRegions Included {} Candidate {}, subsetSize {}, "
- + "maxAllowedSum {}",
- includedRacksOrRegions, rackOrRegion, subsetSize, maxAllowedSum);
- }
+
+ log.trace()
+ .attr("includedRacksOrRegions", includedRacksOrRegions)
+ .attr("rackOrRegion", rackOrRegion)
+ .attr("subsetSize", subsetSize)
+ .attr("maxAllowedSum", maxAllowedSum)
+ .log("CHECK FAILED: RacksOrRegions");
+
return false;
} else {
Set remainingElements = new HashSet(remainingRacksOrRegions);
@@ -276,11 +279,13 @@ public boolean apply(BookieNode candidate) {
// find sets that contain this candidateRackOrRegion
Integer currentAllocation = allocationToRacksOrRegions.get(candidateRackOrRegion);
if (currentAllocation == null) {
- LOG.info("Detected a region that was not initialized {}", candidateRackOrRegion);
+ log.info()
+ .attr("candidateRackOrRegion", candidateRackOrRegion)
+ .log("Detected a region that was not initialized");
if (candidateRackOrRegion.equals(NetworkTopology.DEFAULT_REGION)) {
- LOG.error("Failed to resolve network location {}", candidate);
+ log.error().attr("candidate", candidate).log("Failed to resolve network location");
} else if (!racksOrRegions.contains(candidateRackOrRegion)) {
- LOG.error("Unknown region detected {}", candidateRackOrRegion);
+ log.error().attr("candidateRackOrRegion", candidateRackOrRegion).log("Unknown region detected");
}
allocationToRacksOrRegions.put(candidateRackOrRegion, 0);
currentAllocation = 0;
@@ -563,8 +568,10 @@ public List resolve(List names) {
if (rNames != null && rNames.size() == names.size()) {
for (int i = 0; i < rNames.size(); ++i) {
if (rNames.get(i) == null) {
- LOG.warn("Failed to resolve network location for {}, using default rack for it : {}.",
- names.get(i), defaultRack);
+ log.warn()
+ .attr("bookie", names.get(i))
+ .attr("defaultRack", defaultRack)
+ .log("Failed to resolve network location, using default rack");
failedToResolveNetworkLocationCounter.inc();
rNames.set(i, defaultRack);
}
@@ -572,8 +579,10 @@ public List resolve(List names) {
return rNames;
}
- LOG.warn("Failed to resolve network location for {}, using default rack for them : {}.", names,
- defaultRack);
+ log.warn()
+ .attr("names", names)
+ .attr("defaultRack", defaultRack)
+ .log("Failed to resolve network location, using default rack");
rNames = new ArrayList<>(names.size());
for (int i = 0; i < names.size(); ++i) {
@@ -658,10 +667,13 @@ public Set onClusterChanged(Set writableBookies,
joinedBookies = Sets.difference(writableBookies, oldBookieSet).immutableCopy();
// dead bookies.
deadBookies = Sets.difference(leftBookies, readOnlyBookies).immutableCopy();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : left bookies are {}, joined bookies are {}, while dead bookies are {}.",
- leftBookies, joinedBookies, deadBookies);
- }
+
+ log.debug()
+ .attr("leftBookies", leftBookies)
+ .attr("joinedBookies", joinedBookies)
+ .attr("deadBookies", deadBookies)
+ .log("Cluster changed");
+
handleBookiesThatLeft(leftBookies);
handleBookiesThatJoined(joinedBookies);
if (this.isWeighted && (leftBookies.size() > 0 || joinedBookies.size() > 0)) {
@@ -693,12 +705,15 @@ public void handleBookiesThatLeft(Set leftBookies) {
bookiesLeftCounter.registerSuccessfulValue(1L);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : bookie {} left from cluster.", addr);
- }
+
+ log.debug().attr("bookieAddr", addr).log("Cluster changed: bookie left cluster");
+
}
} catch (Throwable t) {
- LOG.error("Unexpected exception while handling leaving bookie {}", addr, t);
+ log.error()
+ .attr("bookieAddr", addr)
+ .exception(t)
+ .log("Unexpected exception while handling leaving bookie");
if (bookiesLeftCounter != null) {
bookiesLeftCounter.registerFailedValue(1L);
}
@@ -726,12 +741,15 @@ public void handleBookiesThatJoined(Set joinedBookies) {
bookiesJoinedCounter.registerSuccessfulValue(1L);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Cluster changed : bookie {} joined the cluster.", addr);
- }
+
+ log.debug().attr("bookieAddr", addr).log("Cluster changed: bookie joined the cluster");
+
} catch (Throwable t) {
// topology.add() throws unchecked exception
- LOG.error("Unexpected exception while handling joining bookie {}", addr, t);
+ log.error()
+ .attr("bookieAddr", addr)
+ .exception(t)
+ .log("Unexpected exception while handling joining bookie");
bookiesJoinedCounter.registerFailedValue(1L);
// no need to re-throw; we want to process the rest of the bookies
@@ -758,7 +776,10 @@ public void onBookieRackChange(List bookieAddressList) {
}
}
} catch (IllegalArgumentException | NetworkTopologyImpl.InvalidTopologyException e) {
- LOG.error("Failed to update bookie rack info: {} ", bookieAddress, e);
+ log.error()
+ .exception(e)
+ .attr("bookieAddr", bookieAddress)
+ .log("Failed to update bookie rack info");
}
});
} finally {
@@ -785,7 +806,7 @@ public static int differBetweenBookies(List bookiesA, List b
@Override
public void updateBookieInfo(Map bookieInfoMap) {
if (!isWeighted) {
- LOG.info("bookieFreeDiskInfo callback called even without weighted placement policy being used.");
+ log.info("bookieFreeDiskInfo callback called even without weighted placement policy being used.");
return;
}
rwLock.writeLock().lock();
@@ -825,8 +846,11 @@ protected String resolveNetworkLocation(BookieId addr) {
return historyBookie.getNetworkLocation();
}
String defaultRack = getDefaultRack();
- LOG.error("Cannot resolve bookieId {} to a network address, resolving as {}. {}", addr,
- defaultRack, err.getMessage());
+ log.error()
+ .attr("bookieAddr", addr)
+ .attr("defaultRack", defaultRack)
+ .exceptionMessage(err)
+ .log("Cannot resolve bookieId to a network address");
return defaultRack;
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java
index fb01843e64b..27303de0075 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java
@@ -19,23 +19,21 @@
import io.netty.buffer.ByteBuf;
import java.util.List;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.ReadLastConfirmedOp.LastConfirmedDataCallback;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.proto.BookieClient;
import org.apache.bookkeeper.proto.BookieProtocol;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
import org.apache.bookkeeper.proto.checksum.DigestManager.RecoveryData;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* This op is try to read last confirmed without involving quorum coverage checking.
* Use {@link ReadLastConfirmedOp} if you need quorum coverage checking.
*/
+@CustomLog
class TryReadLastConfirmedOp implements ReadEntryCallback {
- static final Logger LOG = LoggerFactory.getLogger(TryReadLastConfirmedOp.class);
-
final LedgerHandle lh;
final BookieClient bookieClient;
final LastConfirmedDataCallback cb;
@@ -67,20 +65,27 @@ public void initiate() {
@Override
public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffer, Object ctx) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("TryReadLastConfirmed received response for (lid={}, eid={}) : {}",
- ledgerId, entryId, rc);
- }
+
+ log.trace()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("rc", rc)
+ .log("TryReadLastConfirmed received response");
+
int bookieIndex = (Integer) ctx;
numResponsesPending--;
if (BKException.Code.OK == rc) {
try {
RecoveryData recoveryData = lh.macManager.verifyDigestAndReturnLastConfirmed(buffer);
- if (LOG.isTraceEnabled()) {
- LOG.trace("Received lastAddConfirmed (lac={}, length={}) from bookie({}) for (lid={}).",
- recoveryData.getLastAddConfirmed(), recoveryData.getLength(), bookieIndex, ledgerId);
- }
+
+ log.trace()
+ .attr("lastAddConfirmed", recoveryData.getLastAddConfirmed())
+ .attr("length", recoveryData.getLength())
+ .attr("bookieIndex", bookieIndex)
+ .attr("ledgerId", ledgerId)
+ .log("Received lastAddConfirmed");
+
if (recoveryData.getLastAddConfirmed() > maxRecoveredData.getLastAddConfirmed()) {
maxRecoveredData = recoveryData;
// callback immediately
@@ -88,9 +93,11 @@ public void readEntryComplete(int rc, long ledgerId, long entryId, ByteBuf buffe
}
hasValidResponse = true;
} catch (BKException.BKDigestMatchException e) {
- LOG.error("Mac mismatch for ledger: " + ledgerId + ", entry: " + entryId
- + " while reading last entry from bookie: "
- + currentEnsemble.get(bookieIndex));
+ log.error()
+ .attr("ledgerId", ledgerId)
+ .attr("entryId", entryId)
+ .attr("bookieAddr", currentEnsemble.get(bookieIndex))
+ .log("Mac mismatch while reading last entry from bookie");
}
} else if (BKException.Code.UnauthorizedAccessException == rc && !completed) {
cb.readLastConfirmedDataComplete(rc, maxRecoveredData);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
index bcdb20464ad..17b3e5c2409 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/UpdateLedgerOp.java
@@ -33,20 +33,18 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
+import lombok.CustomLog;
import org.apache.bookkeeper.bookie.BookieShell.UpdateLedgerNotifier;
import org.apache.bookkeeper.client.api.LedgerMetadata;
import org.apache.bookkeeper.meta.LedgerManager;
import org.apache.bookkeeper.net.BookieId;
import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Encapsulates updating the ledger metadata operation.
*/
+@CustomLog
public class UpdateLedgerOp {
-
- private static final Logger LOG = LoggerFactory.getLogger(UpdateLedgerOp.class);
private final LedgerManager lm;
private final BookKeeperAdmin admin;
@@ -119,11 +117,14 @@ public void updateBookieIdInLedgers(final BookieId oldBookieId, final BookieId n
&& !(ex instanceof BKException.BKNoSuchLedgerExistsOnMetadataServerException)) {
String error = String.format("Failed to update ledger metadata %s, replacing %s with %s",
ledgerId, oldBookieId, newBookieId);
- LOG.error(error, ex);
+ log.error().exception(ex).log(error);
finalPromise.completeExceptionally(new IOException(error, ex));
} else {
- LOG.info("Updated ledger {} metadata, replacing {} with {}",
- ledgerId, oldBookieId, newBookieId);
+ log.info()
+ .attr("ledgerId", ledgerId)
+ .attr("oldBookieId", oldBookieId)
+ .attr("newBookieId", newBookieId)
+ .log("Updated ledger metadata, replaced old bookie with new bookie");
updatedLedgerCnt.incrementAndGet();
progressable.progress(updatedLedgerCnt.get(), issuedLedgerCnt.get());
@@ -144,16 +145,22 @@ public void updateBookieIdInLedgers(final BookieId oldBookieId, final BookieId n
try {
finalPromise.get();
- LOG.info("Total number of ledgers issued={} updated={}",
- issuedLedgerCnt.get(), updatedLedgerCnt.get());
+ log.info()
+ .attr("issued", issuedLedgerCnt.get())
+ .attr("updated", updatedLedgerCnt.get())
+ .log("Total number of ledgers");
} catch (ExecutionException e) {
- String error = String.format("Error waiting for ledger metadata updates to complete (replacing %s with %s)",
- oldBookieId, newBookieId);
- LOG.info(error, e);
+ log.info()
+ .exception(e)
+ .attr("oldBookieId", oldBookieId)
+ .attr("newBookieId", newBookieId)
+ .log("Error waiting for ledger metadata updates to complete");
if (e.getCause() instanceof IOException) {
throw (IOException) e.getCause();
} else {
- throw new IOException(error, e);
+ throw new IOException(
+ String.format("Error waiting for ledger metadata updates to complete (replacing %s with %s)",
+ oldBookieId, newBookieId), e);
}
}
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
index 1a2b9f0dcd5..bbb036f3e69 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
@@ -28,11 +28,10 @@
import java.util.TreeMap;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import lombok.CustomLog;
+@CustomLog
class WeightedRandomSelectionImpl implements WeightedRandomSelection {
- static final Logger LOG = LoggerFactory.getLogger(WeightedRandomSelectionImpl.class);
Double randomMax;
int maxProbabilityMultiplier;
@@ -94,9 +93,12 @@ public int compare(WeightedObject o1, WeightedObject o2) {
medianWeight = median / (double) totalWeight;
minWeight = (double) min / totalWeight;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Updating weights map. MediaWeight: {} MinWeight: {}", medianWeight, minWeight);
- }
+
+ log.debug()
+ .attr("medianWeight", medianWeight)
+ .attr("minWeight", minWeight)
+ .log("Updating weights map");
+
double maxWeight = maxProbabilityMultiplier * medianWeight;
Map weightMap = new HashMap();
@@ -109,10 +111,13 @@ public int compare(WeightedObject o1, WeightedObject o2) {
}
if (maxWeight > 0 && weightedProbability > maxWeight) {
weightedProbability = maxWeight;
- if (LOG.isDebugEnabled()) {
- LOG.debug("Capping the probability to {} for {} Value: {}",
- weightedProbability, e.getKey(), e.getValue());
- }
+
+ log.debug()
+ .attr("weightedProbability", weightedProbability)
+ .attr("key", e.getKey())
+ .attr("value", e.getValue())
+ .log("Capping the probability");
+
}
weightMap.put(e.getKey(), weightedProbability);
}
@@ -124,10 +129,14 @@ public int compare(WeightedObject o1, WeightedObject o2) {
Double key = 0.0;
for (Map.Entry e : weightMap.entrySet()) {
tmpCumulativeMap.put(key, e.getKey());
- if (LOG.isDebugEnabled()) {
- LOG.debug("Key: {} Value: {} AssignedKey: {} AssignedWeight: {}",
- e.getKey(), e.getValue(), key, e.getValue());
- }
+
+ log.debug()
+ .attr("key", e.getKey())
+ .attr("value", e.getValue())
+ .attr("assignedKey", key)
+ .attr("assignedValue", e.getValue())
+ .log("Assigned weight");
+
key += e.getValue();
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java
index 1ce04c4be31..1cccad34804 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ZoneawareEnsemblePlacementPolicyImpl.java
@@ -44,6 +44,7 @@
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.ReentrantReadWriteLock;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
import org.apache.bookkeeper.common.util.ReflectionUtils;
import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -65,16 +66,13 @@
import org.apache.bookkeeper.stats.annotations.StatsDoc;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Simple zoneaware ensemble placement policy.
*/
+@CustomLog
public class ZoneawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsemblePlacementPolicy {
- static final Logger LOG = LoggerFactory.getLogger(ZoneawareEnsemblePlacementPolicyImpl.class);
-
public static final String UNKNOWN_ZONE = "UnknownZone";
/*
* this defaultFaultDomain is used as placeholder network location for
@@ -210,18 +208,21 @@ public Integer getSample() {
if (this.isWeighted) {
this.maxWeightMultiple = conf.getBookieMaxWeightMultipleForWeightBasedPlacement();
this.weightedSelection = new DynamicWeightedRandomSelectionImpl(this.maxWeightMultiple);
- LOG.info("Weight based placement with max multiple of {}", this.maxWeightMultiple);
+ log.info()
+ .attr("maxWeightMultiple", this.maxWeightMultiple)
+ .log("Weight based placement enabled");
} else {
- LOG.info("Not weighted");
+ log.info("Not weighted");
}
this.minNumZonesPerWriteQuorum = conf.getMinNumZonesPerWriteQuorum();
this.desiredNumZonesPerWriteQuorum = conf.getDesiredNumZonesPerWriteQuorum();
this.enforceStrictZoneawarePlacement = conf.getEnforceStrictZoneawarePlacement();
if (minNumZonesPerWriteQuorum > desiredNumZonesPerWriteQuorum) {
- LOG.error(
- "It is misconfigured, for ZoneawareEnsemblePlacementPolicy, minNumZonesPerWriteQuorum: {} cann't be"
- + " greater than desiredNumZonesPerWriteQuorum: {}",
- minNumZonesPerWriteQuorum, desiredNumZonesPerWriteQuorum);
+ log.error()
+ .attr("minNumZonesPerWriteQuorum", minNumZonesPerWriteQuorum)
+ .attr("desiredNumZonesPerWriteQuorum", desiredNumZonesPerWriteQuorum)
+ .log("It is misconfigured, for ZoneawareEnsemblePlacementPolicy,"
+ + " minNumZonesPerWriteQuorum cannot be greater than desiredNumZonesPerWriteQuorum");
throw new IllegalArgumentException("minNumZonesPerWriteQuorum: " + minNumZonesPerWriteQuorum
+ " cann't be greater than desiredNumZonesPerWriteQuorum: " + desiredNumZonesPerWriteQuorum);
}
@@ -251,11 +252,14 @@ public Integer getSample() {
myNode = createDummyLocalBookieNode(InetAddress.getLocalHost().getHostAddress());
myZone = getZoneAwareNodeLocation(myNode).getZone();
} catch (IOException e) {
- LOG.error("Failed to get local host address : ", e);
+ log.error().exception(e).log("Failed to get local host address");
throw new RuntimeException(e);
}
- LOG.info("Initialized zoneaware ensemble placement policy @ {} @ {} : {}.", myNode,
- myNode.getNetworkLocation(), dnsResolver.getClass().getName());
+ log.info()
+ .attr("myNode", myNode)
+ .attr("getNetworkLocation", myNode.getNetworkLocation())
+ .attr("getName", dnsResolver.getClass().getName())
+ .log("Initialized zoneaware ensemble placement policy");
slowBookies = CacheBuilder.newBuilder()
.expireAfterWrite(conf.getBookieFailureHistoryExpirationMSec(), TimeUnit.MILLISECONDS)
@@ -273,7 +277,7 @@ public ZoneawareEnsemblePlacementPolicyImpl withDefaultFaultDomain(String defaul
String[] parts = StringUtils.split(NodeBase.normalize(defaultFaultDomain), NodeBase.PATH_SEPARATOR);
if (parts.length != 2) {
- LOG.error("provided defaultFaultDomain: {} is not valid", defaultFaultDomain);
+ log.error().attr("defaultFaultDomain", defaultFaultDomain).log("provided defaultFaultDomain: is not valid");
throw new IllegalArgumentException("invalid defaultFaultDomain");
} else {
unresolvedNodeLocation = new ZoneAwareNodeLocation(NodeBase.PATH_SEPARATOR_STR + parts[0],
@@ -359,7 +363,7 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
*
* So prohibiting this combination of configuration.
*/
- LOG.error("It is illegal for ensembleSize to be not multiple of"
+ log.error("It is illegal for ensembleSize to be not multiple of"
+ " writeQuorumSize When StrictZoneawarePlacement is enabled");
throw new IllegalArgumentException("It is illegal for ensembleSize to be not multiple of"
+ " writeQuorumSize When StrictZoneawarePlacement is enabled");
@@ -382,7 +386,7 @@ public PlacementResult> newEnsemble(int ensembleSize, int writeQu
*
* So prohibiting this combination of configuration.
*/
- LOG.error("It is illegal for writeQuorumSize to be lesser than or equal"
+ log.error("It is illegal for writeQuorumSize to be lesser than or equal"
+ " to minNumZonesPerWriteQuorum When StrictZoneawarePlacement is enabled");
throw new IllegalArgumentException("It is illegal for writeQuorumSize to be lesser than or equal"
+ " to minNumZonesPerWriteQuorum When StrictZoneawarePlacement is enabled");
@@ -444,7 +448,9 @@ private PlacementResult> createNewEnsembleRandomly(List
int ensembleSize = newEnsemble.size();
Set bookiesToConsider = getBookiesToConsider(excludeBookies);
if (bookiesToConsider.size() < newEnsemble.size()) {
- LOG.error("Not enough bookies are available to form ensemble of size: {}", newEnsemble.size());
+ log.error()
+ .attr("size", newEnsemble.size())
+ .log("Not enough bookies are available to form ensemble");
throw new BKNotEnoughBookiesException();
}
@@ -466,7 +472,7 @@ private PlacementResult selectBookieRandomly(List newEnsembl
int bookieToReplaceIndex = newEnsemble.indexOf(bookieToReplace);
if (bookiesToConsider.isEmpty()) {
- LOG.error("There is no bookie available to replace a bookie");
+ log.error("There is no bookie available to replace a bookie");
throw new BKNotEnoughBookiesException();
}
BookieId candidateAddr = (selectCandidateNode(bookiesToConsider)).getAddr();
@@ -561,8 +567,11 @@ private BookieId setBookieInTheEnsemble(int ensembleSize, int writeQuorumSize,
writeQuorumSize, currentEnsemble, bookieToReplaceIndex, excludeBookies, zonesToExclude);
}
if (bookiesToConsiderAfterExcludingZonesAndUDs.isEmpty()) {
- LOG.error("Not enough bookies are available to replaceBookie : {} in ensemble : {} with excludeBookies {}.",
- bookieToReplace, currentEnsemble, excludeBookies);
+ log.error()
+ .attr("bookieToReplace", bookieToReplace)
+ .attr("currentEnsemble", currentEnsemble)
+ .attr("excludeBookies", excludeBookies)
+ .log("Not enough bookies are available to replaceBookie");
throw new BKNotEnoughBookiesException();
}
@@ -583,7 +592,10 @@ protected Set addDefaultFaultDomainBookies(Set excludeBookie
if (node instanceof BookieNode) {
comprehensiveExclusionBookiesSet.add(((BookieNode) node).getAddr());
} else {
- LOG.error("found non-BookieNode: {} as leaf of defaultFaultDomain: {}", node, getDefaultFaultDomain());
+ log.error()
+ .attr("node", node)
+ .attr("defaultFaultDomain", getDefaultFaultDomain())
+ .log("Found non-BookieNode as leaf of defaultFaultDomain");
}
}
return comprehensiveExclusionBookiesSet;
@@ -872,21 +884,24 @@ public PlacementPolicyAdherence isEnsembleAdheringToPlacementPolicy(List= desiredNumZonesPerWriteQuorumForThisEnsemble) {
if (!validateMinUDsAreMaintained(numOfBookiesInZones, bookiesLocationInWriteSet)) {
placementPolicyAdherence = PlacementPolicyAdherence.FAIL;
- if (LOG.isDebugEnabled()) {
- LOG.debug("in ensemble: {}, writeset starting at: {} doesn't maintain min of 2 UDs"
- + " when there are multiple bookies from the same zone.", ensembleList, i);
- }
+
+ log.debug()
+ .attr("ensembleList", ensembleList)
+ .attr("i", i)
+ .log("in ensemble: writeset starting at: doesn't maintain min of 2 UDs"
+ + " when there are multiple bookies from the same zone.");
+
return placementPolicyAdherence;
}
} else {
if (!validateMinUDsAreMaintained(numOfBookiesInZones, bookiesLocationInWriteSet)) {
placementPolicyAdherence = PlacementPolicyAdherence.FAIL;
- if (LOG.isDebugEnabled()) {
- LOG.debug("in ensemble: {}, writeset starting at: {} doesn't maintain min of 2 UDs"
- + " when there are multiple bookies from the same zone.", ensembleList, i);
- }
+
+ log.debug()
+ .attr("ensembleList", ensembleList)
+ .attr("i", i)
+ .log("in ensemble: writeset starting at: doesn't maintain min of 2 UDs"
+ + " when there are multiple bookies from the same zone.");
+
return placementPolicyAdherence;
}
if (placementPolicyAdherence == PlacementPolicyAdherence.MEETS_STRICT) {
@@ -984,14 +1010,14 @@ public boolean areAckedBookiesAdheringToPlacementPolicy(Set ackedBooki
}
areAckedBookiesAdheringToPlacementPolicy = ((zonesOfAckedBookies
.size() >= minNumZonesPerWriteQuorumForThisEnsemble) && (ackedBookies.size() >= ackQuorumSize));
- if (LOG.isDebugEnabled()) {
- LOG.debug(
- "areAckedBookiesAdheringToPlacementPolicy returning {}, because number of ackedBookies = {},"
- + " number of Zones of ackedbookies = {},"
- + " number of minNumZonesPerWriteQuorumForThisEnsemble = {}",
- areAckedBookiesAdheringToPlacementPolicy, ackedBookies.size(), zonesOfAckedBookies.size(),
- minNumZonesPerWriteQuorumForThisEnsemble);
- }
+
+ log.debug()
+ .attr("areAckedBookiesAdheringToPlacementPolicy", areAckedBookiesAdheringToPlacementPolicy)
+ .attr("size", ackedBookies.size())
+ .attr("size", zonesOfAckedBookies.size())
+ .attr("minNumZonesPerWriteQuorumForThisEnsemble", minNumZonesPerWriteQuorumForThisEnsemble)
+ .log("areAckedBookiesAdheringToPlacementPolicy");
+
} finally {
readLock.unlock();
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java
index 270f115d84b..f61575329cb 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/OpenBuilderBase.java
@@ -19,18 +19,17 @@
package org.apache.bookkeeper.client.impl;
import java.util.Arrays;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.LedgerHandle;
import org.apache.bookkeeper.client.api.BKException.Code;
import org.apache.bookkeeper.client.api.DigestType;
import org.apache.bookkeeper.client.api.OpenBuilder;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Base class for open builders which does the mundane builder stuff.
*/
+@CustomLog
public abstract class OpenBuilderBase implements OpenBuilder {
- static final Logger LOG = LoggerFactory.getLogger(OpenBuilderBase.class);
protected boolean recovery = false;
protected long ledgerId = LedgerHandle.INVALID_LEDGER_ID;
@@ -63,7 +62,7 @@ public OpenBuilder withDigestType(DigestType digestType) {
protected int validate() {
if (ledgerId < 0) {
- LOG.error("invalid ledgerId {} < 0", ledgerId);
+ log.error().attr("ledgerId", ledgerId).log("invalid ledgerId < 0");
return Code.NoSuchLedgerExistsOnMetadataServerException;
}
return Code.OK;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java
index 4a1ad27bb5e..9775c38465d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractHierarchicalLedgerManager.java
@@ -23,6 +23,7 @@
import java.util.TreeSet;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.atomic.AtomicInteger;
+import lombok.CustomLog;
import org.apache.bookkeeper.conf.AbstractConfiguration;
import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.Processor;
import org.apache.bookkeeper.util.StringUtils;
@@ -30,16 +31,13 @@
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code;
import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* An abstract class for managing hierarchical ledgers.
*/
+@CustomLog
public abstract class AbstractHierarchicalLedgerManager extends AbstractZkLedgerManager {
- private static final Logger LOG = LoggerFactory.getLogger(AbstractHierarchicalLedgerManager.class);
-
/**
* Constructor.
*
@@ -67,8 +65,10 @@ public void processResult(int rc, String path, Object ctx) {
finalCb.processResult(successRc, null, context);
return;
} else if (rc != Code.OK.intValue()) {
- LOG.error("Error syncing path " + path + " when getting its children: ",
- KeeperException.create(KeeperException.Code.get(rc), path));
+ log.error()
+ .attr("path", path)
+ .exception(KeeperException.create(KeeperException.Code.get(rc), path))
+ .log("Error syncing path when getting its children");
finalCb.processResult(failureRc, null, context);
return;
}
@@ -82,8 +82,10 @@ public void processResult(int rc, String path, Object ctx,
finalCb.processResult(successRc, null, context);
return;
} else if (rc != Code.OK.intValue()) {
- LOG.error("Error polling hash nodes of " + path,
- KeeperException.create(KeeperException.Code.get(rc), path));
+ log.error()
+ .attr("path", path)
+ .exception(KeeperException.create(KeeperException.Code.get(rc), path))
+ .log("Error polling hash nodes");
finalCb.processResult(failureRc, null, context);
return;
}
@@ -190,7 +192,10 @@ protected NavigableSet ledgerListToSet(List ledgerNodes, String pa
NavigableSet zkActiveLedgers = new TreeSet();
if (!path.startsWith(ledgerRootPath)) {
- LOG.warn("Ledger path [{}] is not a valid path name, it should start with {}", path, ledgerRootPath);
+ log.warn()
+ .attr("path", path)
+ .attr("ledgerRootPath", ledgerRootPath)
+ .log("Ledger path is not a valid path name");
return zkActiveLedgers;
}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
index 59b17be922b..3a2d53a06d6 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java
@@ -33,6 +33,7 @@
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
+import lombok.CustomLog;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.LedgerMetadataBuilder;
import org.apache.bookkeeper.client.api.LedgerMetadata;
@@ -62,16 +63,13 @@
import org.apache.zookeeper.ZooKeeper;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
/**
* Abstract ledger manager based on zookeeper, which provides common methods such as query zk nodes.
*/
+@CustomLog
public abstract class AbstractZkLedgerManager implements LedgerManager, Watcher {
- private static final Logger LOG = LoggerFactory.getLogger(AbstractZkLedgerManager.class);
-
@VisibleForTesting
static final int ZK_CONNECT_BACKOFF_MS = 200;
@@ -100,13 +98,11 @@ protected class ReadLedgerMetadataTask implements Runnable {
@Override
public void run() {
if (null != listeners.get(ledgerId)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Re-read ledger metadata for {}.", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Re-read ledger metadata");
readLedgerMetadata(ledgerId, AbstractZkLedgerManager.this)
.whenComplete((metadata, exception) -> handleMetadata(metadata, exception));
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("Ledger metadata listener for ledger {} is already removed.", ledgerId);
+ } else {
+ log.debug().attr("ledgerId", ledgerId).log("Ledger metadata listener is already removed");
}
}
@@ -114,9 +110,10 @@ private void handleMetadata(Versioned result, Throwable exceptio
if (exception == null) {
final Set listenerSet = listeners.get(ledgerId);
if (null != listenerSet) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Ledger metadata is changed for {} : {}.", ledgerId, result);
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("metadata", result)
+ .log("Ledger metadata is changed");
scheduler.submit(() -> {
synchronized (listenerSet) {
for (LedgerMetadataListener listener : listenerSet) {
@@ -130,10 +127,10 @@ private void handleMetadata(Versioned result, Throwable exceptio
// the ledger is removed, do nothing
Set listenerSet = listeners.remove(ledgerId);
if (null != listenerSet) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Removed ledger metadata listener set on ledger {} as its ledger is deleted : {}",
- ledgerId, listenerSet.size());
- }
+ log.debug()
+ .attr("ledgerId", ledgerId)
+ .attr("listenerCount", listenerSet.size())
+ .log("Removed ledger metadata listener set as ledger is deleted");
// notify `null` as indicator that a ledger is deleted
// make this behavior consistent with `NodeDeleted` watched event.
synchronized (listenerSet) {
@@ -143,8 +140,10 @@ private void handleMetadata(Versioned result, Throwable exceptio
}
}
} else {
- LOG.warn("Failed on read ledger metadata of ledger {}: {}",
- ledgerId, BKException.getExceptionCode(exception));
+ log.warn()
+ .attr("ledgerId", ledgerId)
+ .attr("errorCode", BKException.getExceptionCode(exception))
+ .log("Failed on read ledger metadata");
scheduler.schedule(this, ZK_CONNECT_BACKOFF_MS, TimeUnit.MILLISECONDS);
}
}
@@ -165,9 +164,7 @@ protected class CancelWatchLedgerMetadataTask implements Runnable {
public void run() {
Set listeners = AbstractZkLedgerManager.this.listeners.get(ledgerId);
if (!CollectionUtils.isEmpty(listeners)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Still watch ledgerId: {}, ignore this unwatch task.", ledgerId);
- }
+ log.debug().attr("ledgerId", ledgerId).log("Still watching ledger, ignore this unwatch task");
return;
}
cancelMetadataWatch(ledgerId, AbstractZkLedgerManager.this);
@@ -189,9 +186,7 @@ protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) {
this.ledgerRootPath = ZKMetadataDriverBase.resolveZkLedgersRootPath(conf);
this.scheduler = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("ZkLedgerManagerScheduler"));
- if (LOG.isDebugEnabled()) {
- LOG.debug("Using AbstractZkLedgerManager with root path : {}", ledgerRootPath);
- }
+ log.debug().attr("ledgerRootPath", ledgerRootPath).log("Using AbstractZkLedgerManager");
}
/**
@@ -215,16 +210,14 @@ protected AbstractZkLedgerManager(AbstractConfiguration conf, ZooKeeper zk) {
@Override
public void process(WatchedEvent event) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Received watched event {} from zookeeper based ledger manager.", event);
- }
+ log.debug().attr("event", event).log("Received watched event from zookeeper based ledger manager");
if (Event.EventType.None == event.getType()) {
if (Event.KeeperState.Expired == event.getState()) {
- LOG.info("ZooKeeper client expired on ledger manager.");
+ log.info("ZooKeeper client expired on ledger manager.");
Set keySet = new HashSet