apache / bookkeeper

Apache BookKeeper - a scalable, fault tolerant and low latency storage service optimized for append-only workloads
https://bookkeeper.apache.org/
Apache License 2.0
1.91k stars 904 forks source link

Bookie OutOfDirectMemoryError #3497

Open wolfstudy opened 2 years ago

wolfstudy commented 2 years ago

BUG REPORT

Describe the bug

18:30:13.148 [bookie-io-1-6] ERROR org.apache.bookkeeper.proto.BookieServer - Unable to allocate memory, exiting bookie
io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 4194304 byte(s) of direct memory (used: 68719476735, max: 68719476736)
        at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:802) ~[io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:731) ~[io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:649) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:624) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:203) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena.tcacheAllocateNormal(PoolArena.java:187) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena.allocate(PoolArena.java:136) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PoolArena.allocate(PoolArena.java:126) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:396) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188) ~[io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:163) [org.apache.bookkeeper-bookkeeper-common-allocator-4.14.4.220622-20220622.101608-2.jar:4.14.4.220622-SNAPSHOT]
        at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:157) [org.apache.bookkeeper-bookkeeper-common-allocator-4.14.4.220622-20220622.101608-2.jar:4.14.4.220622-SNAPSHOT]
        at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188) [io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179) [io.netty-netty-buffer-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.channel.unix.PreferredDirectByteBufAllocator.ioBuffer(PreferredDirectByteBufAllocator.java:53) [io.netty-netty-transport-native-unix-common-4.1.76.Final-linux-x86_64.jar:4.1.76.Final]
        at io.netty.channel.DefaultMaxMessagesRecvByteBufAllocator$MaxMessageHandle.allocate(DefaultMaxMessagesRecvByteBufAllocator.java:120) [io.netty-netty-transport-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.channel.epoll.EpollRecvByteAllocatorHandle.allocate(EpollRecvByteAllocatorHandle.java:75) [io.netty-netty-transport-classes-epoll-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:785) [io.netty-netty-transport-classes-epoll-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:487) [io.netty-netty-transport-classes-epoll-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:385) [io.netty-netty-transport-classes-epoll-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:986) [io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) [io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
        at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.76.Final.jar:4.1.76.Final]
        at java.lang.Thread.run(Thread.java:748) [?:1.8.0_232]

18:30:13.148 [bookie-io-1-6] ERROR org.apache.bookkeeper.common.component.ComponentStarter - Triggered exceptionHandler of Component: bookie-server because of Exception in Thread: Thread[bookie-io-1-6,5,main]

Bookie Version: 4.14.4

wolfstudy commented 2 years ago

Unfortunately, we did not observe in this exception stack, where exactly the call caused Bookie to directly overflow the memory. In a production environment, this happens all of a sudden with a running Bookie. And the ledgerStorageClass is follow:

ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage

DirectMemory as follows:

image
dlg99 commented 2 years ago

Most likely the problem is related to the backpressure (or lack of it). Under heavy load Bookie cannot process data fast enough and data is accumulated in memory.

There are two sides of the backpressure configuration:

Ideally, both sides have to be configured. By default the backpressure is disabled.

bookie server needs:

maxAddsInProgressLimit = ..
maxReadsInProgressLimit = ..
closeChannelOnResponseTimeout = true
waitTimeoutOnResponseBackpressureMs = ..

and client:

waitTimeoutOnBackpressureMillis = ..

Pulsar's configuration may need special prefixes (bookkeeper_).

Other things to consider: Run Autorecovery as a separate service (not as part of bookie).

wolfstudy commented 2 years ago

Most likely the problem is related to the backpressure (or lack of it). Under heavy load Bookie cannot process data fast enough and data is accumulated in memory.

There are two sides of the backpressure configuration:

Ideally, both sides have to be configured. By default the backpressure is disabled.

bookie server needs:

maxAddsInProgressLimit = ..
maxReadsInProgressLimit = ..
closeChannelOnResponseTimeout = true
waitTimeoutOnResponseBackpressureMs = ..

and client:

waitTimeoutOnBackpressureMillis = ..

Pulsar's configuration may need special prefixes (bookkeeper_).

Other things to consider: Run Autorecovery as a separate service (not as part of bookie).

Thanks @dlg99 's suggestion. Looking at the monitoring of JVM DirectMemory, it seems that the direct memory has leaked. We monitored the direct memory for 30 days and found that the direct memory grows exponentially and has not been released until it grows to the size of the maximum configured DirectMemory and then OOM occurs. . So I would like to further confirm whether there is a possibility of memory leak anywhere, because the exception stack does not output any information related to the code call stack, so I am not sure whether there are other good ways to locate here.

Future plans to add Netty-related memory leak checking to see if we can find where the memory leak occurs.