Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
2.0.0
-
None
-
None
Description
Master is hung up unable to RPC out to the cluster. It is failing with the below:
Caused by: java.net.SocketTimeoutException: callTimeout=60000, callDuration=69047: Call to ve0801.XYZ.com/10.10.10.10:22101 failed on local exception: java.io.IOException: org.apache.hbase.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 4143972700, max: 4151836672) row '' on table 'hbase:meta' at region=hbase:meta,,1.1588230740, hostname=ve0801.halxg.cloudera.com,22101,1529611440163, seqNum=-1 at org.apache.hadoop.hbase.client.RpcRetryingCallerImpl.callWithRetries(RpcRetryingCallerImpl.java:159) at org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:80) ... 3 more Caused by: java.io.IOException: Call to ve0801.XYZ.com/10.10.10.10:22101 failed on local exception: java.io.IOException: org.apache.hbase.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 4143972700, max: 4151836672) at org.apache.hadoop.hbase.ipc.IPCUtil.wrapException(IPCUtil.java:180) at org.apache.hadoop.hbase.ipc.AbstractRpcClient.onCallFinished(AbstractRpcClient.java:390) at org.apache.hadoop.hbase.ipc.AbstractRpcClient.access$100(AbstractRpcClient.java:95) at org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:410) at org.apache.hadoop.hbase.ipc.AbstractRpcClient$3.run(AbstractRpcClient.java:406) at org.apache.hadoop.hbase.ipc.Call.callComplete(Call.java:103) at org.apache.hadoop.hbase.ipc.Call.setException(Call.java:118) at org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.cleanupCalls(NettyRpcDuplexHandler.java:202) at org.apache.hadoop.hbase.ipc.NettyRpcDuplexHandler.exceptionCaught(NettyRpcDuplexHandler.java:219) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:285) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:264) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireExceptionCaught(AbstractChannelHandlerContext.java:256) at org.apache.hbase.thirdparty.io.netty.channel.ChannelInboundHandlerAdapter.exceptionCaught(ChannelInboundHandlerAdapter.java:131) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeExceptionCaught(AbstractChannelHandlerContext.java:285) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.notifyHandlerException(AbstractChannelHandlerContext.java:850) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:364) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) at org.apache.hbase.thirdparty.io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:286) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:340) at org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1359) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:348) at org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:935) at org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:801) at org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe$1.run(AbstractEpollChannel.java:412) at org.apache.hbase.thirdparty.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163) at org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:403) at org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:309) at org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858) at org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) ... 1 more Caused by: java.io.IOException: org.apache.hbase.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 4143972700, max: 4151836672) at org.apache.hadoop.hbase.ipc.IPCUtil.toIOE(IPCUtil.java:148) ... 26 more Caused by: org.apache.hbase.thirdparty.io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 4143972700, max: 4151836672) at org.apache.hbase.thirdparty.io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:640) at org.apache.hbase.thirdparty.io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:594) at org.apache.hbase.thirdparty.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:764) at org.apache.hbase.thirdparty.io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:740) at org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:244) at org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.allocate(PoolArena.java:226) at org.apache.hbase.thirdparty.io.netty.buffer.PoolArena.reallocate(PoolArena.java:397) at org.apache.hbase.thirdparty.io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:118) at org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBuf.ensureWritable0(AbstractByteBuf.java:285) at org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:265) at org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:1077) at org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:1070) at org.apache.hbase.thirdparty.io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:1060) at org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder$1.cumulate(ByteToMessageDecoder.java:92) at org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:263) at org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:362)
Perhaps we are leaking – fun, fun, see https://github.com/jeffgriffith/native-jvm-leaks – or perhaps we actually need 4G of offheap running a Master for 650 nodes and 300k regions.
Filing this issue as our first foray into the wonderful world of offheap accounting and tracking.
Attachments
1.
|
Trim Master memory usage | Resolved | Michael Stack |