You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Xiaolin Ha (Jira)" <ji...@apache.org> on 2021/11/18 03:59:00 UTC

[jira] [Comment Edited] (HBASE-26460) Close netty channel causes regionserver crash in handleTooBigRequest

    [ https://issues.apache.org/jira/browse/HBASE-26460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17445620#comment-17445620 ] 

Xiaolin Ha edited comment on HBASE-26460 at 11/18/21, 3:58 AM:
---------------------------------------------------------------

> For ByteBuf, netty uses reference counting to manager its lifecycle, so I think even if we call release when closing the channel, we should still have a reference in the ipc handler if it is still in use?

[~zhangduo] I think we hope to have reference in the ipc handler, but actually not according current codes.

The RPC call object doesn't use the Netty ReferenceCounted object to increase the Netty byte buffer relevant reference count, it is just a higher level and dependent count, it is an AtomicInteger. Netty will not recognize this count. For Netty, the byte buffer reference count is just increased ONCE after NettyRpcFrameDecoder by readRetainedSlice(). And for normal requests, it is released once when the reference count recorded in the ServerCall decreased to 0.

 


was (Author: xiaolin ha):
> For ByteBuf, netty uses reference counting to manager its lifecycle, so I think even if we call release when closing the channel, we should still have a reference in the ipc handler if it is still in use?

[~zhangduo] I think we hope to have reference in the ipc handler, but actually not according current codes.

The RPC call object doesn't use the Netty ReferenceCounted object to increase the Netty byte buffer relevant reference count, it is just a higher level and dependent count, it is an

AtomicInteger. Netty will not recognize this count. For Netty, the byte buffer reference count is just increased ONCE after NettyRpcFrameDecoder by readRetainedSlice(). And for normal requests, it is released once when the reference count recorded in the ServerCall decreased to 0.

 

> Close netty channel causes regionserver crash in handleTooBigRequest
> --------------------------------------------------------------------
>
>                 Key: HBASE-26460
>                 URL: https://issues.apache.org/jira/browse/HBASE-26460
>             Project: HBase
>          Issue Type: Bug
>          Components: rpc
>    Affects Versions: 3.0.0-alpha-1, 2.0.0
>            Reporter: Xiaolin Ha
>            Assignee: Xiaolin Ha
>            Priority: Critical
>
> In HBASE-26170, I proposed the coredump problem after calling handleTooBigRequest, but that issue did not resolve the regionserver crash problem, which occurs before the WAL corruption in HBASE-24984.
> After looking through the codes, I think the problem is in CLOSE channel. 
> The direct byte buffer used by RPC call request is allocated by Netty, though we add a reference count to record when to release the direct byte buffer, the byte buffer is managed by Netty actually. It is allocated from Netty PoolArena, and is released there. 
> When the HBase ipc handler is processing a request, the Netty channel handler can process the channel events and message coming back in succession. When there is a too big request by NettyRpcFrameDecoder, the channel will be closed, and all the resources of the channel will be released, though there is HBase ipc handlers using the direct byte buffer to process previous requests.
> Netty provides two methods to request the pooled byte buffer, one is through the PoolThreadCache, each handler thread owns a private one. Another is through PoolArena#allocateNormal. Each ChannelHandler has a local PoolThreadCache.
> When a new Netty channel is created, a new ChannelHandler instance is created. 
> And when a channel is closed, the relevant channel handler will be removed from the pipeline. I found this annotation in the Channel class of Netty,
> {code:java}
> It is important to call close() or close(ChannelPromise) to release all resources once you are done with the Channel. This ensures all resources are released in a proper way, i.e. filehandles. {code}
> And when channel handler is removed in ByteToMessageDecoder#handlerRemoved, it will release the byte buffer,
> {code:java}
> @Override
> public final void handlerRemoved(ChannelHandlerContext ctx) throws Exception {
>     if (decodeState == STATE_CALLING_CHILD_DECODE) {
>         decodeState = STATE_HANDLER_REMOVED_PENDING;
>         return;
>     }
>     ByteBuf buf = cumulation;
>     if (buf != null) {
>         // Directly set this to null so we are sure we not access it in any other method here anymore.
>         cumulation = null;
>         int readable = buf.readableBytes();
>         if (readable > 0) {
>             ByteBuf bytes = buf.readBytes(readable);
>             buf.release();
>             ctx.fireChannelRead(bytes);
>         } else {
>             buf.release();
>         }
> ... {code}
> We should not close the channel when encountering too big request, I think it should just skip the bytes like that in LengthFieldBasedFrameDecoder.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)