chenxu created HBASE-23699:
------------------------------

             Summary: ByteBuff may not recycle correctly when NettyRpcServer 
enabled and client disconnect
                 Key: HBASE-23699
                 URL: https://issues.apache.org/jira/browse/HBASE-23699
             Project: HBase
          Issue Type: Bug
          Components: rpc
            Reporter: chenxu
            Assignee: chenxu


Code in NettyRpcServerResponseEncoder
{code:java}
@Override
public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise 
promise) throws Exception {
  if (msg instanceof RpcResponse) {
    RpcResponse resp = (RpcResponse) msg;
    BufferChain buf = resp.getResponse();
    ctx.write(Unpooled.wrappedBuffer(buf.getBuffers()), promise).addListener(f 
-> {
      resp.done();
      if (f.isSuccess()) {
        metrics.sentBytes(buf.size());
      }
    });
  } else {
    ctx.write(msg, promise);
  }
}
{code}
If exception occurs during ChannelHandlerContext#write (such as Client 
disconnected), ByteBuff may not be recycled correctly.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to