Description
Time taken for read with 4 datanodes (in EC acceptance test) increased from around ~1.5 to over 6 minutes due to closePipeline call introduced in HDDS-9151.
SCM log shows delays of 30 seconds in stale/dead node handlers:
2023-08-24 06:16:32,459 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Datanode 9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) moved to stale state. Finalizing its pipelines [PipelineID=34069f4a-8d02-4832-b876-9e0f2d28955f, PipelineID=8445e3a4-cb66-461d-be67-8e2f0b4b364a, PipelineID=b7094ae1-845d-4cac-bf80-005e348df5ce, PipelineID=8e1fb491-ba5b-4715-b4bf-d1d88f40f462, PipelineID=5d21f513-bef0-4cd7-a12d-3b6b9a8b4e72, PipelineID=dd9faf5e-c7c4-4fba-8f32-5e8abca97dd5, PipelineID=4a670212-d8e3-44d3-910e-55de8e42a689] 2023-08-24 06:17:02,461 [EventQueue-DeadNodeForDeadNodeHandler] INFO node.DeadNodeHandler: A dead datanode is detected. 9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:17:02,466 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=34069f4a-8d02-4832-b876-9e0f2d28955f for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:17:32,468 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=8445e3a4-cb66-461d-be67-8e2f0b4b364a for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:18:02,470 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=b7094ae1-845d-4cac-bf80-005e348df5ce for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:18:02,485 [EventQueue-DeadNodeForDeadNodeHandler] WARN node.DeadNodeHandler: Exception while finalizing pipeline PipelineID=34069f4a-8d02-4832-b876-9e0f2d28955f 2023-08-24 06:18:32,472 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=8e1fb491-ba5b-4715-b4bf-d1d88f40f462 for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:18:32,489 [EventQueue-DeadNodeForDeadNodeHandler] WARN node.DeadNodeHandler: Exception while finalizing pipeline PipelineID=8445e3a4-cb66-461d-be67-8e2f0b4b364a 2023-08-24 06:19:02,474 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=5d21f513-bef0-4cd7-a12d-3b6b9a8b4e72 for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:20:02,498 [EventQueue-DeadNodeForDeadNodeHandler] WARN node.DeadNodeHandler: Exception while finalizing pipeline PipelineID=b7094ae1-845d-4cac-bf80-005e348df5ce 2023-08-24 06:20:32,499 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=dd9faf5e-c7c4-4fba-8f32-5e8abca97dd5 for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:21:32,519 [EventQueue-StaleNodeForStaleNodeHandler] INFO node.StaleNodeHandler: Could not finalize pipeline=PipelineID=4a670212-d8e3-44d3-910e-55de8e42a689 for dn=9e6aa1b0-c0be-4b45-a28d-138fd69d5679(ozone_datanode_5.ozone_default/172.18.0.10) 2023-08-24 06:21:32,532 [EventQueue-DeadNodeForDeadNodeHandler] WARN node.DeadNodeHandler: Exception while finalizing pipeline PipelineID=8e1fb491-ba5b-4715-b4bf-d1d88f40f462
Stack dump indicates these are waiting (in turn) for a lock held by the StateMachineUpdater thread, which is waiting for completion of a Ratis request with a timeout of 30 seconds.
"86bcc2bb-26a7-44ec-b092-111c6f7e11bd@group-7B0514C5188A-StateMachineUpdater" #65 daemon prio=5 os_prio=0 cpu=118.39ms elapsed=256.36s tid=0x00007f8b1c78a800 nid=0x174 waiting on condition [0x00007f8af14ca000] java.lang.Thread.State: TIMED_WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.19/Native Method) - parking to wait for <0x0000000418efaeb0> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.19/LockSupport.java:234) at java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.19/CompletableFuture.java:1798) at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.19/ForkJoinPool.java:3128) at java.util.concurrent.CompletableFuture.timedGet(java.base@11.0.19/CompletableFuture.java:1868) at java.util.concurrent.CompletableFuture.get(java.base@11.0.19/CompletableFuture.java:2021) at org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl.submitRequest(SCMRatisServerImpl.java:229) at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatisServer(SCMHAInvocationHandler.java:123) at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invokeRatis(SCMHAInvocationHandler.java:112) at org.apache.hadoop.hdds.scm.ha.SCMHAInvocationHandler.invoke(SCMHAInvocationHandler.java:74) at com.sun.proxy.$Proxy17.updatePipelineState(Unknown Source) at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.closePipeline(PipelineManagerImpl.java:524) at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.removeContainerFromPipeline(PipelineManagerImpl.java:416) at org.apache.hadoop.hdds.scm.container.ContainerStateManagerImpl.lambda$getContainerStateChangeActions$0(ContainerStateManagerImpl.java:275) at org.apache.hadoop.hdds.scm.container.ContainerStateManagerImpl$$Lambda$328/0x000000084039e040.accept(Unknown Source) at org.apache.hadoop.hdds.scm.container.ContainerStateManagerImpl.updateContainerState(ContainerStateManagerImpl.java:368) at jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(java.base@11.0.19/Native Method) at jdk.internal.reflect.NativeMethodAccessorImpl.invoke(java.base@11.0.19/NativeMethodAccessorImpl.java:62) at jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(java.base@11.0.19/DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(java.base@11.0.19/Method.java:566) at org.apache.hadoop.hdds.scm.ha.SCMStateMachine.process(SCMStateMachine.java:188) at org.apache.hadoop.hdds.scm.ha.SCMStateMachine.applyTransaction(SCMStateMachine.java:148) at org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:1777) at org.apache.ratis.server.impl.StateMachineUpdater.applyLog(StateMachineUpdater.java:242) at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:184) at java.lang.Thread.run(java.base@11.0.19/Thread.java:829) Locked ownable synchronizers: - <0x0000000444a595c0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) - <0x00000004455de840> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) "EventQueue-DeadNodeForDeadNodeHandler" #409 daemon prio=5 os_prio=0 cpu=2.26ms elapsed=34.85s tid=0x0000559839ca9000 nid=0x27e9 waiting on condition [0x00007f8ad78a2000] java.lang.Thread.State: WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.19/Native Method) - parking to wait for <0x00000004455de840> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) at java.util.concurrent.locks.LockSupport.park(java.base@11.0.19/LockSupport.java:194) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(java.base@11.0.19/AbstractQueuedSynchronizer.java:885) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.base@11.0.19/AbstractQueuedSynchronizer.java:917) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(java.base@11.0.19/AbstractQueuedSynchronizer.java:1240) at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(java.base@11.0.19/ReentrantReadWriteLock.java:959) at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.acquireWriteLock(PipelineManagerImpl.java:943) at org.apache.hadoop.hdds.scm.pipeline.PipelineManagerImpl.closePipeline(PipelineManagerImpl.java:522) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler.lambda$null$0(DeadNodeHandler.java:125) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler$$Lambda$790/0x0000000840718440.accept(Unknown Source) at java.lang.Iterable.forEach(java.base@11.0.19/Iterable.java:75) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler.lambda$destroyPipelines$1(DeadNodeHandler.java:123) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler$$Lambda$789/0x0000000840718040.accept(Unknown Source) at java.util.Optional.ifPresent(java.base@11.0.19/Optional.java:183) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler.destroyPipelines(DeadNodeHandler.java:122) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler.onMessage(DeadNodeHandler.java:83) at org.apache.hadoop.hdds.scm.node.DeadNodeHandler.onMessage(DeadNodeHandler.java:49) at org.apache.hadoop.hdds.server.events.SingleThreadExecutor.lambda$onMessage$1(SingleThreadExecutor.java:85) at org.apache.hadoop.hdds.server.events.SingleThreadExecutor$$Lambda$680/0x0000000840699440.run(Unknown Source) at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.19/ThreadPoolExecutor.java:1128) at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.19/ThreadPoolExecutor.java:628) at java.lang.Thread.run(java.base@11.0.19/Thread.java:829) Locked ownable synchronizers: - <0x000000041b0ed090> (a java.util.concurrent.ThreadPoolExecutor$Worker)