Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-26460

Close netty channel causes regionserver crash in handleTooBigRequest

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Critical
    • Resolution: Unresolved
    • 3.0.0-alpha-1, 2.0.0
    • None
    • rpc
    • None

    Description

      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,

      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. 

      And when channel handler is removed in ByteToMessageDecoder#handlerRemoved, it will release the byte buffer,

      @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();
              }
      ... 

      We should not close the channel when encountering too big request, I think it should just skip the bytes like that in LengthFieldBasedFrameDecoder.

      Attachments

        Activity

          People

            Xiaolin Ha Xiaolin Ha
            Xiaolin Ha Xiaolin Ha
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated: