Skip to content

Commit e80d031

Browse files
authored
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: #3901 #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](#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 #4607. Then it reading with a wrong map size which could take a lot of memory.
1 parent 45f7a92 commit e80d031

File tree

2 files changed

+5
-1
lines changed

2 files changed

+5
-1
lines changed

bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/DefaultEntryLogger.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1061,7 +1061,7 @@ public EntryLogMetadata getEntryLogMetadata(long entryLogId, AbstractLogCompacto
10611061
} catch (FileNotFoundException fne) {
10621062
LOG.warn("Cannot find entry log file {}.log : {}", Long.toHexString(entryLogId), fne.getMessage());
10631063
throw fne;
1064-
} catch (Exception e) {
1064+
} catch (Throwable e) {
10651065
LOG.info("Failed to get ledgers map index from: {}.log : {}", entryLogId, e.getMessage());
10661066

10671067
// Fall-back to scanning

bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -531,6 +531,10 @@ public void runWithFlags(boolean force, boolean suspendMajor, boolean suspendMin
531531
LOG.error("Error in entryLog-metadatamap, Failed to complete GC/Compaction due to entry-log {}",
532532
e.getMessage(), e);
533533
gcStats.getGcThreadRuntime().registerFailedEvent(MathUtils.elapsedNanos(threadStart), TimeUnit.NANOSECONDS);
534+
} catch (Throwable e) {
535+
LOG.error("Error in garbage collector thread, Failed to complete GC/Compaction due to {}",
536+
e.getMessage(), e);
537+
gcStats.getGcThreadRuntime().registerFailedEvent(MathUtils.elapsedNanos(threadStart), TimeUnit.NANOSECONDS);
534538
} finally {
535539
if (force && forceGarbageCollection.compareAndSet(true, false)) {
536540
LOG.info("{} Set forceGarbageCollection to false after force GC to make it forceGC-able again.",

0 commit comments

Comments
 (0)