Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
2.3.4
-
None
-
None
-
None
Description
In our cluster, JVM crash when the request size is too big
hs_err.log
# # A fatal error has been detected by the Java Runtime Environment: # # SIGSEGV (0xb) at pc=0x00007f6c4bbb7b31, pid=5619, tid=0x00007f3dc57b4700 # # JRE version: Java(TM) SE Runtime Environment (8.0_181-b13) (build 1.8.0_181-b13) # Java VM: Java HotSpot(TM) 64-Bit Server VM (25.181-b13 mixed mode linux-amd64 ) # Problematic frame: # C [libc.so.6+0x15bb31] __memmove_ssse3_back+0x1ba1 # # Core dump written. Default location: /home/user/core or core.5619 (max size 1048576 kB). To ensure a full core dump, try "ulimit -c unlimited" before starting Java again # # If you would like to submit a bug report, please visit: # http://bugreport.java.com/bugreport/crash.jsp # Stack: [0x00007f3dc56b4000,0x00007f3dc57b5000], sp=0x00007f3dc57b2d48, free space=1019k Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code) C [libc.so.6+0x15bb31] __memmove_ssse3_back+0x1ba1 J 2301 sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V (0 bytes) @ 0x00007f6c35a3ae21 [0x00007f6c35a3ad40+0xe1] j org.apache.hadoop.hbase.util.UnsafeAccess.unsafeCopy(Ljava/lang/Object;JLjava/lang/Object;JJ)V+36 j org.apache.hadoop.hbase.util.UnsafeAccess.copy(Ljava/nio/ByteBuffer;I[BII)V+69 j org.apache.hadoop.hbase.util.ByteBufferUtils.copyFromBufferToArray([BLjava/nio/ByteBuffer;III)V+39 j org.apache.hadoop.hbase.CellUtil.copyQualifierTo(Lorg/apache/hadoop/hbase/Cell;[BI)I+31 j org.apache.hadoop.hbase.CellUtil.cloneQualifier(Lorg/apache/hadoop/hbase/Cell;)[B+12 j org.apache.hadoop.hbase.ByteBufferKeyValue.getQualifierArray()[B+1 j org.apache.hadoop.hbase.CellUtil.getCellKeyAsString(Lorg/apache/hadoop/hbase/Cell;Ljava/util/function/Function;)Ljava/lang/String;+97 j org.apache.hadoop.hbase.CellUtil.getCellKeyAsString(Lorg/apache/hadoop/hbase/Cell;)Ljava/lang/String;+6 j org.apache.hadoop.hbase.CellUtil.toString(Lorg/apache/hadoop/hbase/Cell;Z)Ljava/lang/String;+16 j org.apache.hadoop.hbase.ByteBufferKeyValue.toString()Ljava/lang/String;+2 j org.apache.hadoop.hbase.client.Mutation.add(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/client/Mutation;+28 j org.apache.hadoop.hbase.client.Put.add(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/client/Put;+2 J 19274 C2 org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toPut(Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MutationProto;Lorg/apache/hadoop/hbase/CellScanner;)Lorg/apache/hadoop/hbase/client/Put; (910 bytes) @ 0x00007f6c386ed4e4 [0x00007f6c386eb7a0+0x1d44] J 32557 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionActionResult$Builder;Lorg/apache/hadoop/hbase/regionserver/HRegion;Lorg/apache/hadoop/hbase/quotas/OperationQuota;Ljava/util/List;Lorg/apache/hadoop/hbase/CellScanner;JLorg/apache/hadoop/hbase/quotas/ActivePolicyEnforcement;Z)V (1046 bytes) @ 0x00007f6c39d9e494 [0x00007f6c39d9dcc0+0x7d4] J 29517 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(Lorg/apache/hadoop/hbase/regionserver/HRegion;Lorg/apache/hadoop/hbase/quotas/OperationQuota;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionAction;Lorg/apache/hadoop/hbase/CellScanner;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionActionResult$Builder;Ljava/util/List;JLorg/apache/hadoop/hbase/regionserver/RSRpcServices$RegionScannersCloseCallBack;Lorg/apache/hadoop/hbase/ipc/RpcCallContext;Lorg/apache/hadoop/hbase/quotas/ActivePolicyEnforcement;)Ljava/util/List; (901 bytes) @ 0x00007f6c39c25898 [0x00007f6c39c24da0+0xaf8] J 31074 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(Lorg/apache/hbase/thirdparty/com/google/protobuf/RpcController;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MultiRequest;)Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MultiResponse; (1119 bytes) @ 0x00007f6c39e7dcd4 [0x00007f6c39e7db20+0x1b4] J 28404 C2 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(Lorg/apache/hbase/thirdparty/com/google/protobuf/Descriptors$MethodDescriptor;Lorg/apache/hbase/thirdparty/com/google/protobuf/RpcController;Lorg/apache/hbase/thirdparty/com/google/protobuf/Message;)Lorg/apache/hbase/thirdparty/com/google/protobuf/Message; (221 bytes) @ 0x00007f6c38ec2008 [0x00007f6c38ec1e00+0x208] J 31063 C2 org.apache.hadoop.hbase.ipc.RpcServer.call(Lorg/apache/hadoop/hbase/ipc/RpcCall;Lorg/apache/hadoop/hbase/monitoring/MonitoredRPCHandler;)Lorg/apache/hadoop/hbase/util/Pair; (634 bytes) @ 0x00007f6c39e6c8e8 [0x00007f6c39e6c0a0+0x848] J 29524 C2 org.apache.hadoop.hbase.ipc.CallRunner.run()V (1376 bytes) @ 0x00007f6c39c5e3f8 [0x00007f6c39c5da00+0x9f8] J 20025 C2 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(Lorg/apache/hadoop/hbase/ipc/CallRunner;)V (268 bytes) @ 0x00007f6c36f98918 [0x00007f6c36f987a0+0x178] J 20980% C2 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run()V (72 bytes) @ 0x00007f6c37ee2698 [0x00007f6c37ee1e80+0x818] v ~StubRoutines::call_stub V [libjvm.so+0x697a76] JavaCalls::call_helper(JavaValue*, methodHandle*, JavaCallArguments*, Thread*)+0x1056 V [libjvm.so+0x697f81] JavaCalls::call_virtual(JavaValue*, KlassHandle, Symbol*, Symbol*, JavaCallArguments*, Thread*)+0x321 V [libjvm.so+0x698427] JavaCalls::call_virtual(JavaValue*, Handle, KlassHandle, Symbol*, Symbol*, Thread*)+0x47 V [libjvm.so+0x71789e] thread_entry(JavaThread*, Thread*)+0x7e V [libjvm.so+0xa813f3] JavaThread::thread_main_inner()+0x103 V [libjvm.so+0xa8153c] JavaThread::run()+0x11c V [libjvm.so+0x930198] java_start(Thread*)+0x108 C [libpthread.so.0+0x7e65] start_thread+0xc5 Java frames: (J=compiled Java code, j=interpreted, Vv=VM code) J 2301 sun.misc.Unsafe.copyMemory(Ljava/lang/Object;JLjava/lang/Object;JJ)V (0 bytes) @ 0x00007f6c35a3ada7 [0x00007f6c35a3ad40+0x67] j org.apache.hadoop.hbase.util.UnsafeAccess.unsafeCopy(Ljava/lang/Object;JLjava/lang/Object;JJ)V+36 j org.apache.hadoop.hbase.util.UnsafeAccess.copy(Ljava/nio/ByteBuffer;I[BII)V+69 j org.apache.hadoop.hbase.util.ByteBufferUtils.copyFromBufferToArray([BLjava/nio/ByteBuffer;III)V+39 j org.apache.hadoop.hbase.CellUtil.copyQualifierTo(Lorg/apache/hadoop/hbase/Cell;[BI)I+31 j org.apache.hadoop.hbase.CellUtil.cloneQualifier(Lorg/apache/hadoop/hbase/Cell;)[B+12 j org.apache.hadoop.hbase.ByteBufferKeyValue.getQualifierArray()[B+1 j org.apache.hadoop.hbase.CellUtil.getCellKeyAsString(Lorg/apache/hadoop/hbase/Cell;Ljava/util/function/Function;)Ljava/lang/String;+97 j org.apache.hadoop.hbase.CellUtil.getCellKeyAsString(Lorg/apache/hadoop/hbase/Cell;)Ljava/lang/String;+6 j org.apache.hadoop.hbase.CellUtil.toString(Lorg/apache/hadoop/hbase/Cell;Z)Ljava/lang/String;+16 j org.apache.hadoop.hbase.ByteBufferKeyValue.toString()Ljava/lang/String;+2 j org.apache.hadoop.hbase.client.Mutation.add(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/client/Mutation;+28 j org.apache.hadoop.hbase.client.Put.add(Lorg/apache/hadoop/hbase/Cell;)Lorg/apache/hadoop/hbase/client/Put;+2 J 19274 C2 org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.toPut(Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MutationProto;Lorg/apache/hadoop/hbase/CellScanner;)Lorg/apache/hadoop/hbase/client/Put; (910 bytes) @ 0x00007f6c386ed4e4 [0x00007f6c386eb7a0+0x1d44] J 32557 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp(Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionActionResult$Builder;Lorg/apache/hadoop/hbase/regionserver/HRegion;Lorg/apache/hadoop/hbase/quotas/OperationQuota;Ljava/util/List;Lorg/apache/hadoop/hbase/CellScanner;JLorg/apache/hadoop/hbase/quotas/ActivePolicyEnforcement;Z)V (1046 bytes) @ 0x00007f6c39d9e494 [0x00007f6c39d9dcc0+0x7d4] J 29517 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation(Lorg/apache/hadoop/hbase/regionserver/HRegion;Lorg/apache/hadoop/hbase/quotas/OperationQuota;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionAction;Lorg/apache/hadoop/hbase/CellScanner;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$RegionActionResult$Builder;Ljava/util/List;JLorg/apache/hadoop/hbase/regionserver/RSRpcServices$RegionScannersCloseCallBack;Lorg/apache/hadoop/hbase/ipc/RpcCallContext;Lorg/apache/hadoop/hbase/quotas/ActivePolicyEnforcement;)Ljava/util/List; (901 bytes) @ 0x00007f6c39c25898 [0x00007f6c39c24da0+0xaf8] J 31074 C2 org.apache.hadoop.hbase.regionserver.RSRpcServices.multi(Lorg/apache/hbase/thirdparty/com/google/protobuf/RpcController;Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MultiRequest;)Lorg/apache/hadoop/hbase/shaded/protobuf/generated/ClientProtos$MultiResponse; (1119 bytes) @ 0x00007f6c39e7dcd4 [0x00007f6c39e7db20+0x1b4] J 28404 C2 org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(Lorg/apache/hbase/thirdparty/com/google/protobuf/Descriptors$MethodDescriptor;Lorg/apache/hbase/thirdparty/com/google/protobuf/RpcController;Lorg/apache/hbase/thirdparty/com/google/protobuf/Message;)Lorg/apache/hbase/thirdparty/com/google/protobuf/Message; (221 bytes) @ 0x00007f6c38ec2008 [0x00007f6c38ec1e00+0x208] J 31063 C2 org.apache.hadoop.hbase.ipc.RpcServer.call(Lorg/apache/hadoop/hbase/ipc/RpcCall;Lorg/apache/hadoop/hbase/monitoring/MonitoredRPCHandler;)Lorg/apache/hadoop/hbase/util/Pair; (634 bytes) @ 0x00007f6c39e6c8e8 [0x00007f6c39e6c0a0+0x848] J 29524 C2 org.apache.hadoop.hbase.ipc.CallRunner.run()V (1376 bytes) @ 0x00007f6c39c5e3f8 [0x00007f6c39c5da00+0x9f8] J 20025 C2 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(Lorg/apache/hadoop/hbase/ipc/CallRunner;)V (268 bytes) @ 0x00007f6c36f98918 [0x00007f6c36f987a0+0x178] J 20980% C2 org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run()V (72 bytes) @ 0x00007f6c37ee2698 [0x00007f6c37ee1e80+0x818] v ~StubRoutines::call_stub
regionserver log
2022-12-21 20:38:03,341 WARN [RS-EventLoopGroup-1-67] ipc.NettyRpcServer: RPC data length of 741092396 received from 11.0.10.10 is greater than max allowed 268435456. Set "hbase.ipc.max.request.size" on server to override this limit (not recommended) 2022-12-21 20:38:54,426 INFO [main] zookeeper.ZooKeeper: Client environment:zookeeper.version=3.5.7-f0fdd52973d373ffd9c86b81d99842dc2c7f660e, built on 02/10/2020 11:30 GMT 2022-12-21 20:38:54,426 INFO [main] zookeeper.ZooKeeper: Client environment:host.name=HT-HBASE-xxxx-10-70-121-0.hadoop.jd.local 2022-12-21 20:38:54,426 INFO [main] zookeeper.ZooKeeper: Client environment:java.version=1.8.0_181 2022-12-21 20:38:54,427 INFO [main] zookeeper.ZooKeeper: Client environment:java.vendor=Oracle Corporation 2022-12-21 20:38:54,427 INFO [main] zookeeper.ZooKeeper: Client environment:java.home=/software/servers/jdk1.8.0_181/jre
May be the Bytebuffer is already release.
Attachments
Issue Links
- duplicates
-
HBASE-25997 NettyRpcFrameDecoder decode request header wrong when handleTooBigRequest
- Resolved
-
HBASE-26170 handleTooBigRequest in NettyRpcServer didn't skip enough bytes
- Resolved