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

Andrew Kyle Purtell edited comment on HBASE-26708 at 6/9/22 11:15 PM:
----------------------------------------------------------------------

On the subject of configuration and NettyRpcServer, we leave netty level 
resource limits unbounded. The number of threads to use for the event loop is 
default 0 (unbounded). The default for io.netty.eventLoop.maxPendingTasks is 
INT_MAX. We don't do this for our own RPC handlers. We have a notion of maximum 
handler pool size, with a default of 30, typically raised in production by the 
user. We constrain the depth of the request queue in multiple ways... limits on 
number of queued calls, limits on total size of calls data that can be queued 
(to avoid memory usage overrun, just like this case), CoDel conditioning of the 
call queues if it is enabled, and so on.

Under load can we pile up a excess of pending request state, such as direct 
buffers containing request bytes, at the netty layer because of downstream 
resource limits? Those limits will act as a bottleneck, as intended, and before 
would have also applied backpressure through RPC too, because SimpleRpcServer 
had thread limits ("hbase.ipc.server.read.threadpool.size", default 10), but 
now netty may be able to queue up a lot more, in comparison, because netty has 
been designed for concurrency. 

This is going to be somewhat application dependent too. If the application 
interacts synchronously with calls and has its own bound, then in flight 
requests or their network level handling will be bounded by the aggregate 
(client_in_flight_max x number_of_clients). If the application is highly async, 
write-mostly, or a load test client – which is typically write-mostly, async, 
and configured with large bounds :) – then this can explain the findings 
reported here. It may also explain why security makes it worse, because when 
security is active we wrap (encrypt) and unwrap (decrypt) up in the call layer, 
beyond netty, and that takes additional time there, which would back things up 
at the netty layer more than if call handling would complete more quickly 
without encryption.

Consider the hbase.netty.eventloop.rpcserver.thread.count default. It is 0 
(unbounded). I don't know what it can actually get up to in production, because 
we lack the metric, but there are diminishing returns when threads > cores so a 
reasonable default here could be Runtime.getRuntime().availableProcessors() 
instead of unbounded?

maxPendingTasks probably should not be INT_MAX, but that may matter less.

The goal would be to limit concurrency at the netty layer in such a way that:
1. Performance is still good
2. Under load, we don't balloon resource usage at the netty layer

I could be looking at something that isn't the real issue but it is notable.


was (Author: apurtell):
On the subject of configuration and NettyRpcServer, we leave netty level 
resource limits unbounded. The number of threads to use for the event loop is 
default 0 (unbounded). The default for io.netty.eventLoop.maxPendingTasks is 
INT_MAX. We don't do this for our own RPC handlers. We have a notion of maximum 
handler pool size, with a default of 30, typically raised in production by the 
user. We constrain the depth of the request queue in multiple ways... limits on 
number of queued calls, limits on total size of calls data that can be queued 
(to avoid memory usage overrun, just like this case), CoDel conditioning of the 
call queues if it is enabled, and so on.

Under load can we pile up a excess of pending request state, such as direct 
buffers containing request bytes, at the netty layer because of downstream 
resource limits? Those limits will act as a bottleneck, as intended, and before 
would have also applied backpressure through RPC too, because SimpleRpcServer 
had thread limits ("hbase.ipc.server.read.threadpool.size", default 10), but 
now netty may be able to queue up a lot more, in comparison, because netty has 
been designed for concurrency. 

This is going to be somewhat application dependent too. If the application 
interacts synchronously with calls and has its own bound, then in flight 
requests or their network level handling will be bounded by the aggregate 
(client_limit x number_of_clients). If the application is highly async, 
write-mostly, or a load test client – which is typically write-mostly, async, 
and configured with large bounds :) – then this can explain the findings 
reported here. It may also explain why security makes it worse, because when 
security is active we wrap (encrypt) and unwrap (decrypt) up in the call layer, 
beyond netty, and that takes additional time there, which would back things up 
at the netty layer more than if call handling would complete more quickly 
without encryption.

Consider the hbase.netty.eventloop.rpcserver.thread.count default. It is 0 
(unbounded). I don't know what it can actually get up to in production, because 
we lack the metric, but there are diminishing returns when threads > cores so a 
reasonable default here could be Runtime.getRuntime().availableProcessors() 
instead of unbounded?

maxPendingTasks probably should not be INT_MAX, but that may matter less.

The goal would be to limit concurrency at the netty layer in such a way that:
1. Performance is still good
2. Under load, we don't balloon resource usage at the netty layer

I could be looking at something that isn't the real issue but it is notable.

> Netty "leak detected" and OutOfDirectMemoryError due to direct memory 
> buffering
> -------------------------------------------------------------------------------
>
>                 Key: HBASE-26708
>                 URL: https://issues.apache.org/jira/browse/HBASE-26708
>             Project: HBase
>          Issue Type: Bug
>          Components: rpc
>    Affects Versions: 2.5.0, 2.4.6
>            Reporter: Viraj Jasani
>            Priority: Critical
>
> Under constant data ingestion, using default Netty based RpcServer and 
> RpcClient implementation results in OutOfDirectMemoryError, supposedly caused 
> by leaks detected by Netty's LeakDetector.
> {code:java}
> 2022-01-25 17:03:10,084 ERROR [S-EventLoopGroup-1-3] 
> util.ResourceLeakDetector - java:115)
>   
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.expandCumulation(ByteToMessageDecoder.java:538)
>   
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder$1.cumulate(ByteToMessageDecoder.java:97)
>   
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:274)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
>   
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>   
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:795)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:480)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
>   
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
>   
> org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>   
> org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   java.lang.Thread.run(Thread.java:748)
>  {code}
> {code:java}
> 2022-01-25 17:03:14,014 ERROR [S-EventLoopGroup-1-3] 
> util.ResourceLeakDetector - 
> apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:507)
>   
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:446)
>   
> org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)
>   
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)
>   
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)
>   
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:795)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:480)
>   
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
>   
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
>   
> org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>   
> org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   java.lang.Thread.run(Thread.java:748)
>  {code}
> And finally handlers are removed from the pipeline due to 
> OutOfDirectMemoryError:
> {code:java}
> 2022-01-25 17:36:28,657 WARN  [S-EventLoopGroup-1-5] 
> channel.DefaultChannelPipeline - An exceptionCaught() event was fired, and it 
> reached at the tail of the pipeline. It usually means the last handler in the 
> pipeline did not handle the exception.
> org.apache.hbase.thirdparty.io.netty.channel.ChannelPipelineException: 
> org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler.handlerAdded()
>  has thrown an exception; removed.
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.callHandlerAdded0(DefaultChannelPipeline.java:624)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.addFirst(DefaultChannelPipeline.java:181)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.addFirst(DefaultChannelPipeline.java:358)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.addFirst(DefaultChannelPipeline.java:339)
>   at 
> org.apache.hadoop.hbase.ipc.NettyRpcConnection.saslNegotiate(NettyRpcConnection.java:229)
>   at 
> org.apache.hadoop.hbase.ipc.NettyRpcConnection.access$600(NettyRpcConnection.java:79)
>   at 
> org.apache.hadoop.hbase.ipc.NettyRpcConnection$2.operationComplete(NettyRpcConnection.java:312)
>   at 
> org.apache.hadoop.hbase.ipc.NettyRpcConnection$2.operationComplete(NettyRpcConnection.java:300)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:578)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:571)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:550)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:491)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:616)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.setSuccess0(DefaultPromise.java:605)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:104)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:84)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:653)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:691)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:470)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>   at java.lang.Thread.run(Thread.java:748)
> Caused by: 
> org.apache.hbase.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: 
> failed to allocate 16777216 byte(s) of direct memory (used: 33269220801, max: 
> 33285996544)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:802)
>   at 
> org.apache.hbase.thirdparty.io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:731)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:632)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:607)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:202)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.tcacheAllocateSmall(PoolArena.java:172)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.allocate(PoolArena.java:134)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.allocate(PoolArena.java:126)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:395)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:178)
>   at 
> org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBufAllocator.buffer(AbstractByteBufAllocator.java:115)
>   at 
> org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler.writeResponse(NettyHBaseSaslRpcClientHandler.java:79)
>   at 
> org.apache.hadoop.hbase.security.NettyHBaseSaslRpcClientHandler.handlerAdded(NettyHBaseSaslRpcClientHandler.java:115)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.callHandlerAdded(AbstractChannelHandlerContext.java:938)
>   at 
> org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.callHandlerAdded0(DefaultChannelPipeline.java:609)
>   ... 24 more
>  {code}



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to