This is an automated email from the ASF dual-hosted git repository.
yong pushed a commit to branch branch-4.17
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git
The following commit(s) were added to refs/heads/branch-4.17 by this push:
new 55308e90c5 Log all the error in the GarbageCollectorThread (#4649)
55308e90c5 is described below
commit 55308e90c52cdee5b3f4ebb63502d41d855b69b1
Author: Yong Zhang <[email protected]>
AuthorDate: Tue Aug 12 10:18:44 2025 +0800
Log all the error in the GarbageCollectorThread (#4649)
### Motivation
We met the GarbageCollectionThread was stopped by some runtime error, but
we didn't catch it then, causing the GC to stop.
Such as:
https://github.com/apache/bookkeeper/pull/3901
https://github.com/apache/bookkeeper/pull/4544
In our case, the GC stopped because of the OutOfDirectMemoryException then
the process stopped and the files can not be deleted. But we didn't see any
error logs. This PR enhance the log info when an unhandled error happens.
We already have the [PR](https://github.com/apache/bookkeeper/pull/4544)
fixed that.
And another fix in this PR is to change the Exception to the Throwable in
the getEntryLogMetadata.
Here is the error stack:
```
io.netty.util.internal.OutOfDirectMemoryError: failed to allocate
213909504 byte(s) of direct memory (used: 645922847, max: 858783744)
at
io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:880)
at
io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:809)
at
io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:718)
at
io.netty.buffer.PoolArena$DirectArena.newUnpooledChunk(PoolArena.java:707)
at io.netty.buffer.PoolArena.allocateHuge(PoolArena.java:224)
at io.netty.buffer.PoolArena.allocate(PoolArena.java:142)
at io.netty.buffer.PoolArena.reallocate(PoolArena.java:317)
at io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:123)
at
io.netty.buffer.AbstractByteBuf.ensureWritable0(AbstractByteBuf.java:305)
at
io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:280)
at
io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:1103)
at
org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:104)
at
org.apache.bookkeeper.bookie.DefaultEntryLogger.extractEntryLogMetadataFromIndex(DefaultEntryLogger.java:1109)
at
org.apache.bookkeeper.bookie.DefaultEntryLogger.getEntryLogMetadata(DefaultEntryLogger.java:1060)
at
org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:678)
at
org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:365)
at
org.apache.bookkeeper.bookie.GarbageCollectorThread.lambda$triggerGC$4(GarbageCollectorThread.java:268)
at
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
at java.base/java.util.concurrent.FutureTask.run(Unknown Source)
at
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown
Source)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Unknown Source)
```
You can see it get much more memory used here
extractEntryLogMetadataFromIndex(DefaultEntryLogger.java:1109). The reason is
that the header has the wrong data of the header, which should already be fixed
by https://github.com/apache/bookkeeper/pull/4607. Then it reading with a wrong
map size which could take a lot of memory.
(cherry picked from commit e80d0318cfdebbc79f37d9120de8df28c8c1c13a)
---
.../main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java | 2 +-
.../java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java | 4 ++++
2 files changed, 5 insertions(+), 1 deletion(-)
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java
index 34c9f8e7e3..2696aee3c9 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java
@@ -1061,7 +1061,7 @@ public class DefaultEntryLogger implements EntryLogger {
} catch (FileNotFoundException fne) {
LOG.warn("Cannot find entry log file {}.log : {}",
Long.toHexString(entryLogId), fne.getMessage());
throw fne;
- } catch (Exception e) {
+ } catch (Throwable e) {
LOG.info("Failed to get ledgers map index from: {}.log : {}",
entryLogId, e.getMessage());
// Fall-back to scanning
diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
index 3675f62bc9..d1528c8a4f 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
@@ -504,6 +504,10 @@ public class GarbageCollectorThread implements Runnable {
e.getMessage(), e);
gcStats.getGcThreadRuntime().registerFailedEvent(
MathUtils.nowInNano() - threadStart, TimeUnit.NANOSECONDS);
+ } catch (Throwable e) {
+ LOG.error("Error in garbage collector thread, Failed to complete
GC/Compaction due to {}",
+ e.getMessage(), e);
+
gcStats.getGcThreadRuntime().registerFailedEvent(MathUtils.elapsedNanos(threadStart),
TimeUnit.NANOSECONDS);
} finally {
if (force && forceGarbageCollection.compareAndSet(true, false)) {
LOG.info("{} Set forceGarbageCollection to false after force
GC to make it forceGC-able again.",