You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
BUG REPORT
Recently, in our customer product, we found that the bookie direct memory increases slowly over time, finally it throws netty allocate direct memory failed exception, then bookie restart.
Dump info
So we dump the heap and found that there are some unreachable ReadResponse, and the data property ByteBuf refCnt is 2, and the ByteBuf is also unreachable, so there is no chance to release the ByteBuf, it will be deleted after JVM GC.
See the picture, we can think the memory leak already happen.
Log info
In the bookie log, we found there are many disconnect info like:
The Root Case
The memory leak occurred in an edge scene.
In the normal case, we get the data from BookieImpl and wrap it in ReadResponse, and then write the ReadResponse to the netty channel, there is our netty handler to handle the ReadResponse.
The ByteBufList is implemented interce ReferenceCounted, so after netty flush it, netty will release it by invoke io.netty.util.ReferenceCountUtil#release(java.lang.Object)
public static boolean release(Object msg, int decrement) {
ObjectUtil.checkPositive(decrement, "decrement");
if (msg instanceof ReferenceCounted) {
return ((ReferenceCounted) msg).release(decrement);
}
return false;
}
In the io.netty.util.ReferenceCountUtil#release(java.lang.Object), it will check if the msg is implemented ReferenceCounted. If not, do nothing.
There is an important netty mechanism you should know, after a connection disconnect, there are two things that happen.
see line_851, the outboundBuffer is null, it will release ReadResponse, but the ReadResponse didn't implement ReferenceCounted, so it didn't release the ReadResponse.data. Memory leak!!!
To Reproduce
Steps to reproduce the behavior:
Start bookies.
Start a bookkeeper client using V2 protocol.
use the bookkeeper client to read entries.
In the reading process, kill -9 bookkeeper client program.
Bookie server will memory leak.
The text was updated successfully, but these errors were encountered:
BUG REPORT
Recently, in our customer product, we found that the bookie direct memory increases slowly over time, finally it throws netty allocate direct memory failed exception, then bookie restart.
Dump info
So we dump the heap and found that there are some unreachable
ReadResponse
, and the data propertyByteBuf
refCnt is 2, and theByteBuf
is also unreachable, so there is no chance to release theByteBuf
, it will be deleted after JVM GC.See the picture, we can think the memory leak already happen.
Log info
In the bookie log, we found there are many disconnect info like:
The Root Case
The memory leak occurred in an edge scene.
In the normal case, we get the data from BookieImpl and wrap it in ReadResponse, and then write the ReadResponse to the netty channel, there is our netty handler to handle the ReadResponse.
The ByteBufList is implemented interce
ReferenceCounted
, so after netty flush it, netty will release it by invokeio.netty.util.ReferenceCountUtil#release(java.lang.Object)
In the
io.netty.util.ReferenceCountUtil#release(java.lang.Object)
, it will check if the msg is implemented ReferenceCounted. If not, do nothing.There is an important netty mechanism you should know, after a connection disconnect, there are two things that happen.
all the handlers will be removed from the ChannelPipeline after channelUnregistered.
https://github.com/netty/netty/blob/7971075bbe9f5509c8b20c0e702ec2affb37d76e/transport/src/main/java/io/netty/channel/DefaultChannelPipeline.java#L1387-L1394
At line_1392, it will remove all the handlers, only leaving two netty default contextHandler in the ChannelPipeline, HeadContext and TailContext
Make AbstractChannel.AbstractUnsafe#outboundBuffer = null.
https://github.com/netty/netty/blob/7971075bbe9f5509c8b20c0e702ec2affb37d76e/transport/src/main/java/io/netty/channel/AbstractChannel.java#L703
So we know, after the connection disconnect, the channel only left HeadContext and TailContext, and the AbstractChannel.AbstractUnsafe#outboundBuffer is null. If the bookie still write ReadResponse to netty channel, the
ResponseEncoder
didn't work(already removed), the HeadContext will write ReadResponse to AbstractUnsafe#outboundBuffer directly.https://github.com/netty/netty/blob/7971075bbe9f5509c8b20c0e702ec2affb37d76e/transport/src/main/java/io/netty/channel/AbstractChannel.java#L847-L864
see line_851, the outboundBuffer is null, it will release ReadResponse, but the ReadResponse didn't implement ReferenceCounted, so it didn't release the ReadResponse.data. Memory leak!!!
To Reproduce
Steps to reproduce the behavior:
The text was updated successfully, but these errors were encountered: