Details
-
Improvement
-
Status: Open
-
Major
-
Resolution: Unresolved
-
1.4.0
-
None
Description
Datanode read performance degradation due to KeyValueContainer waiting readLock for a long time. Lock competition becomes especially intense during high concurrent write operations, such as large-scale data deletions. This can result in all ChunkReader threads being in waiting for lock state.
The complete jstack as: dn.chunkreader-waiting.jstack
All of the ChunkReader threads are blocked in the `KeyValueHandler#checkContainerIsHealthy` method, waiting for the `KeyValueContainer#readLock`. Here is the stack trace for one of the threads:
"ChunkReader-99" #1948 daemon prio=5 os_prio=0 tid=0x00000000024e5000 nid=0x3ae0b waiting on condition [0x00007fee6cfce000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00007ff7f8797b58> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(AbstractQueuedSynchronizer.java:967) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(AbstractQueuedSynchronizer.java:1283) at java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(ReentrantReadWriteLock.java:727) at org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer.readLock(KeyValueContainer.java:726) at org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.checkContainerIsHealthy(KeyValueHandler.java:753) at org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handleReadChunk(KeyValueHandler.java:707) at org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.dispatchRequest(KeyValueHandler.java:260) at org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler.handle(KeyValueHandler.java:223) at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatchRequest(HddsDispatcher.java:326) at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.lambda$dispatch$0(HddsDispatcher.java:179) at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher$$Lambda$948/1865391148.apply(Unknown Source) at org.apache.hadoop.hdds.server.OzoneProtocolMessageDispatcher.processRequest(OzoneProtocolMessageDispatcher.java:87) at org.apache.hadoop.ozone.container.common.impl.HddsDispatcher.dispatch(HddsDispatcher.java:178) at org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:57) at org.apache.hadoop.ozone.container.common.transport.server.GrpcXceiverService$1.onNext(GrpcXceiverService.java:50) at org.apache.ratis.thirdparty.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:262) at org.apache.ratis.thirdparty.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33) at org.apache.hadoop.hdds.tracing.GrpcServerInterceptor$1.onMessage(GrpcServerInterceptor.java:49) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailableInternal(ServerCallImpl.java:333) at org.apache.ratis.thirdparty.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:316) at org.apache.ratis.thirdparty.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:835) at org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37) at org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Locked ownable synchronizers: - <0x00007ff39d077d88> (a java.util.concurrent.ThreadPoolExecutor$Worker)