[
https://issues.apache.org/jira/browse/HBASE-26460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17445588#comment-17445588
]
Lijin Bin commented on HBASE-26460:
-----------------------------------
I change the rpcserver from NettyRpcServer to SimpleRpcServer and add some log,
i guess the regionserver receive corrupt request.
{code}
2021-10-24 00:30:02,663 WARN
[RpcServer.default.FPBQ.Fifo.handler=388,queue=38,port=60020]
hbase.KeyValueUtil: Invalid key length in KeyValue. keyLength=386442382,
KeyValueBytesHex=\x17\x08\xA4\x8E\x0A\x1A\x05Multi
\x01*\x04\x08\xF5\xF3f8\xF0\x93\x09\x98.\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0011@1624040345@5rj,1634050797652.1e23827e4c16c3567495a08c9b6aa9f2.\x1A2\x08\x9D\x01\x12-\x0A\x1B0013@1635006529@5rn0dsrvp@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x85\x01\x0AO\x08\x01\x12Kweishi_test,0087@1623354672,1634024008759.c8693c6db7c157ad64aa7278eb9bddd3.\x1A2\x08\xA5\x03\x12-\x0A\x1B0087@1635006571@5riigna5j@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0167@1630260990@5rc,1630440425435.717d01827d8e0812ef3bd49e6dd22001.\x1A2\x08\xCF\x01\x12-\x0A\x1B0172@1635006588@5rn0dsq0s@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0349@1623347840@4om,1623688029220.645cc9ac3baad8f......,
offset=5942, length=1691943
2021-10-24 00:30:02,663 WARN
[RpcServer.default.FPBQ.Fifo.handler=388,queue=38,port=60020]
hbase.KeyValueUtil: Invalid value length in KeyValue, valueLength=1970041961,
KeyValueBytesHex=\x0A\x1A\x05Multi
\x01*\x04\x08\xF5\xF3f8\xF0\x93\x09\x98.\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0011@1624040345@5rj,1634050797652.1e23827e4c16c3567495a08c9b6aa9f2.\x1A2\x08\x9D\x01\x12-\x0A\x1B0013@1635006529@5rn0dsrvp@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x85\x01\x0AO\x08\x01\x12Kweishi_test,0087@1623354672,1634024008759.c8693c6db7c157ad64aa7278eb9bddd3.\x1A2\x08\xA5\x03\x12-\x0A\x1B0087@1635006571@5riigna5j@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0167@1630260990@5rc,1630440425435.717d01827d8e0812ef3bd49e6dd22001.\x1A2\x08\xCF\x01\x12-\x0A\x1B0172@1635006588@5rn0dsq0s@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0349@1623347840@4om,1623688029220.645cc9ac3baad8f5e60......,
offset=5946, length=386442382
2021-10-24 00:30:02,663 INFO
[RpcServer.default.FPBQ.Fifo.handler=388,queue=38,port=60020] ipc.RpcServer:
RpcCall is callId: 165623 service: ClientService methodName: Multi size: 1.6 M
connection: 9.18.190.168:39036 deadline: 1635006752662
2021-10-24 00:30:02,664 INFO
[RpcServer.default.FPBQ.Fifo.handler=388,queue=38,port=60020] ipc.RpcServer:
RpcCall is callId: 165623 service: ClientService methodName: Multi size: 1.6 M
connection: 9.18.190.168:39036 deadline: 1635006752662 param: region=
weishi_test,0011@1624040345@5rj,1634050797652.1e23827e4c16c3567495a08c9b6aa9f2.,
for 51 action(s) and 1st row key=0013@1635006529@5rn0dsrvp@1 connection:
9.18.190.168:39036
2021-10-24 00:30:02,664 ERROR
[RpcServer.default.FPBQ.Fifo.handler=388,queue=38,port=60020] ipc.RpcServer:
Unexpected throwable object
java.lang.IllegalArgumentException: Invalid value length in KeyValue,
valueLength=1970041961, KeyValueBytesHex=\x0A\x1A\x05Multi
\x01*\x04\x08\xF5\xF3f8\xF0\x93\x09\x98.\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0011@1624040345@5rj,1634050797652.1e23827e4c16c3567495a08c9b6aa9f2.\x1A2\x08\x9D\x01\x12-\x0A\x1B0013@1635006529@5rn0dsrvp@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x85\x01\x0AO\x08\x01\x12Kweishi_test,0087@1623354672,1634024008759.c8693c6db7c157ad64aa7278eb9bddd3.\x1A2\x08\xA5\x03\x12-\x0A\x1B0087@1635006571@5riigna5j@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0167@1630260990@5rc,1630440425435.717d01827d8e0812ef3bd49e6dd22001.\x1A2\x08\xCF\x01\x12-\x0A\x1B0172@1635006588@5rn0dsq0s@1\x10\x02
\xFF\xFF\xFF\xFF\xFF\xFF\xFF\xFF\x7F0\x00@\x01\x0A\x89\x01\x0AS\x08\x01\x12Oweishi_test,0349@1623347840@4om,1623688029220.645cc9ac3baad8f5e60......,
offset=5946, length=386442382
at
org.apache.hadoop.hbase.KeyValueUtil.checkKeyValueBytes(KeyValueUtil.java:567)
at org.apache.hadoop.hbase.KeyValue.<init>(KeyValue.java:344)
at org.apache.hadoop.hbase.NoTagsKeyValue.<init>(NoTagsKeyValue.java:34)
at
org.apache.hadoop.hbase.codec.KeyValueCodec$ByteBuffKeyValueDecoder.createCell(KeyValueCodec.java:110)
at
org.apache.hadoop.hbase.codec.KeyValueCodec$ByteBuffKeyValueDecoder.advance(KeyValueCodec.java:98)
at
org.apache.hadoop.hbase.regionserver.RSRpcServices.skipCellsForMutation(RSRpcServices.java:2966)
at
org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(RSRpcServices.java:1105)
at
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicBatchOp(RSRpcServices.java:940)
at
org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(RSRpcServices.java:904)
at
org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(RSRpcServices.java:2903)
at
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:45265)
at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:411)
at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:133)
at
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
at
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)
{code}
> 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)