Skip to content

Conversation

@hangc0276
Copy link
Contributor

Motivation

When one ledger file is broken, getEntryLogMetadata will throw an IllegalArgumentException, which is a RuntimeException and won't be caught in the whole garbage collector execution path. The exception will be caught by the thread SafeRunnable and interrupt the garbage collector, leading to those deleted ledgers can't be recycled and the ledger disk usage up.

2023-04-01T00:55:29,497+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Garbage collector thread forced to perform GC before expiry of wait time.
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 17
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index from: 17.log : Cannot deserialize ledgers map from ledger 9062744587808030975
2023-04-01T00:55:29,553+0000 [GarbageCollectorThread-11-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalArgumentException: Negative position
	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) ~[?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
	at java.lang.Thread.run(Thread.java:829) ~[?:?]

Changes

Catch the RuntimeException when getting the metadata for each entry log file.

Copy link
Member

@horizonzy horizonzy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM.

@hangc0276
Copy link
Contributor Author

@eolivelli Would you please take a look? Thanks.

entryLogMetaMap.put(entryLogId, entryLogMeta);
}
} catch (IOException e) {
} catch (IOException | RuntimeException e) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what if the RuntimeException is thrown for a different reason and can't be that easily skipped
I think it makes more sense to make EntryLogger.getEntryLogMetadata throw the IOException in this case

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This try-catch code block reads each entry log file's metadata and checks if this entry log file can be deleted. The RuntimeException will be thrown by the read entry log file's metadata logic. We should block the whole garbage collection thread if one entry log file's metadata read failed. We can skip this entry log file if it is broken.

@xiang092689
Copy link
Contributor

Is the broken EntryLog file will be left in file system forever?

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@hangc0276
Copy link
Contributor Author

Is the broken EntryLog file will be left in file system forever?

@xiang092689 Yes. It will be better than all the entry log files can't be garbage collected.

@hangc0276
Copy link
Contributor Author

@dlg99 Do you have any concerns?

@hangc0276
Copy link
Contributor Author

@dlg99 Do you have any concerns?

@dlg99 If no objections, I will merge this PR.

@hangc0276
Copy link
Contributor Author

@dlg99 Do you have any concerns?

@dlg99 If no objections, I will merge this PR.

Merge it.

@hangc0276 hangc0276 merged commit 8309b18 into apache:master May 15, 2023
zymap pushed a commit that referenced this pull request Jun 19, 2023
### Motivation
When one ledger file is broken, `getEntryLogMetadata` will throw an IllegalArgumentException, which is a RuntimeException and won't be caught in the whole garbage collector execution path. The exception will be caught by the thread SafeRunnable and interrupt the garbage collector, leading to those deleted ledgers can't be recycled and the ledger disk usage up.

```
2023-04-01T00:55:29,497+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Garbage collector thread forced to perform GC before expiry of wait time.
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 17
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index from: 17.log : Cannot deserialize ledgers map from ledger 9062744587808030975
2023-04-01T00:55:29,553+0000 [GarbageCollectorThread-11-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalArgumentException: Negative position
	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) ~[?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
	at java.lang.Thread.run(Thread.java:829) ~[?:?]
```

### Changes
Catch the RuntimeException when getting the metadata for each entry log file.

(cherry picked from commit 8309b18)
hangc0276 added a commit to hangc0276/bookkeeper that referenced this pull request Jun 26, 2023
When one ledger file is broken, `getEntryLogMetadata` will throw an IllegalArgumentException, which is a RuntimeException and won't be caught in the whole garbage collector execution path. The exception will be caught by the thread SafeRunnable and interrupt the garbage collector, leading to those deleted ledgers can't be recycled and the ledger disk usage up.

```
2023-04-01T00:55:29,497+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Garbage collector thread forced to perform GC before expiry of wait time.
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 17
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index from: 17.log : Cannot deserialize ledgers map from ledger 9062744587808030975
2023-04-01T00:55:29,553+0000 [GarbageCollectorThread-11-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalArgumentException: Negative position
	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) ~[?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
	at java.lang.Thread.run(Thread.java:829) ~[?:?]
```

Catch the RuntimeException when getting the metadata for each entry log file.

(cherry picked from commit 8309b18)
zymap pushed a commit that referenced this pull request Dec 6, 2023
### Motivation
When one ledger file is broken, `getEntryLogMetadata` will throw an IllegalArgumentException, which is a RuntimeException and won't be caught in the whole garbage collector execution path. The exception will be caught by the thread SafeRunnable and interrupt the garbage collector, leading to those deleted ledgers can't be recycled and the ledger disk usage up.

```
2023-04-01T00:55:29,497+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Garbage collector thread forced to perform GC before expiry of wait time.
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 17
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index from: 17.log : Cannot deserialize ledgers map from ledger 9062744587808030975
2023-04-01T00:55:29,553+0000 [GarbageCollectorThread-11-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalArgumentException: Negative position
	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) ~[?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
	at java.lang.Thread.run(Thread.java:829) ~[?:?]
```

### Changes
Catch the RuntimeException when getting the metadata for each entry log file.

(cherry picked from commit 8309b18)
Ghatage pushed a commit to sijie/bookkeeper that referenced this pull request Jul 12, 2024
### Motivation
When one ledger file is broken, `getEntryLogMetadata` will throw an IllegalArgumentException, which is a RuntimeException and won't be caught in the whole garbage collector execution path. The exception will be caught by the thread SafeRunnable and interrupt the garbage collector, leading to those deleted ledgers can't be recycled and the ledger disk usage up.

```
2023-04-01T00:55:29,497+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Garbage collector thread forced to perform GC before expiry of wait time.
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.GarbageCollectorThread - Extracting entry log meta from entryLogId: 17
2023-04-01T00:55:29,498+0000 [GarbageCollectorThread-11-1] INFO  org.apache.bookkeeper.bookie.EntryLogger - Failed to get ledgers map index from: 17.log : Cannot deserialize ledgers map from ledger 9062744587808030975
2023-04-01T00:55:29,553+0000 [GarbageCollectorThread-11-1] ERROR org.apache.bookkeeper.common.util.SafeRunnable - Unexpected throwable caught
java.lang.IllegalArgumentException: Negative position
	at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:785) ~[?:?]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:93) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.BufferedReadChannel.read(BufferedReadChannel.java:65) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.readFromLogChannel(EntryLogger.java:418) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.scanEntryLog(EntryLogger.java:996) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.extractEntryLogMetadataByScanning(EntryLogger.java:1136) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.EntryLogger.getEntryLogMetadata(EntryLogger.java:1045) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.extractMetaFromEntryLogs(GarbageCollectorThread.java:607) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.runWithFlags(GarbageCollectorThread.java:348) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.bookie.GarbageCollectorThread.safeRun(GarbageCollectorThread.java:329) ~[org.apache.bookkeeper-bookkeeper-server-4.14.5.jar:4.14.5]
	at org.apache.bookkeeper.common.util.SafeRunnable.run(SafeRunnable.java:36) ~[org.apache.bookkeeper-bookkeeper-common-4.14.5.jar:4.14.5]
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
	at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:305) ~[?:?]
	at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:305) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
	at java.lang.Thread.run(Thread.java:829) ~[?:?]
```

### Changes
Catch the RuntimeException when getting the metadata for each entry log file.
zymap added a commit that referenced this pull request Aug 12, 2025
### 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.
zymap added a commit that referenced this pull request Aug 12, 2025
### 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.

(cherry picked from commit e80d031)
zymap added a commit that referenced this pull request Aug 12, 2025
### 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.

(cherry picked from commit e80d031)
priyanshu-ctds pushed a commit to datastax/bookkeeper that referenced this pull request Aug 20, 2025
### 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:
apache#3901
apache#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](apache#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 apache#4607. Then it reading with a wrong map size which could take a lot of memory.

(cherry picked from commit e80d031)
(cherry picked from commit 55308e9)
srinath-ctds pushed a commit to datastax/bookkeeper that referenced this pull request Aug 21, 2025
### 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:
apache#3901
apache#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](apache#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 apache#4607. Then it reading with a wrong map size which could take a lot of memory.

(cherry picked from commit e80d031)
(cherry picked from commit 55308e9)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

6 participants