[ https://issues.apache.org/jira/browse/NIFI-6696?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Mark Payne updated NIFI-6696: ----------------------------- Status: Patch Available (was: Open) > Deadlock can occur when node disconnects from cluster if using load-balanced > connections > ---------------------------------------------------------------------------------------- > > Key: NIFI-6696 > URL: https://issues.apache.org/jira/browse/NIFI-6696 > Project: Apache NiFi > Issue Type: Bug > Components: Core Framework > Reporter: Mark Payne > Assignee: Mark Payne > Priority: Blocker > Fix For: 1.10.0 > > > I have encountered a deadlock when a node disconnected from the cluster. The > relevant stack traces are as below: > > - Holds SocketLoadBalanceFlowFileQueue.partitionWriteLock > - Needs NioAsyncLoadBalanceClient synchronized > {code:java} > Thread 61318: (state = BLOCKED) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.register(java.lang.String, > java.util.function.BooleanSupplier, java.util.function.Supplier, > org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback, > > org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback, > java.util.function.Supplier, java.util.function.BooleanSupplier) @bci=0, > line=109 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientRegistry.lambda$register$0(java.lang.String, > java.util.function.BooleanSupplier, java.util.function.Supplier, > org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback, > > org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback, > java.util.function.Supplier, java.util.function.BooleanSupplier, > org.apache.nifi.controller.queue.clustered.client.async.AsyncLoadBalanceClient) > @bci=12, line=63 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientRegistry$$Lambda$1143.accept(java.lang.Object) > @bci=32 (Compiled frame) > - java.lang.Iterable.forEach(java.util.function.Consumer) @bci=30, line=75 > (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientRegistry.register(java.lang.String, > org.apache.nifi.cluster.protocol.NodeIdentifier, > java.util.function.BooleanSupplier, java.util.function.Supplier, > org.apache.nifi.controller.queue.clustered.client.async.TransactionFailureCallback, > > org.apache.nifi.controller.queue.clustered.client.async.TransactionCompleteCallback, > java.util.function.Supplier, java.util.function.BooleanSupplier) @bci=46, > line=63 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition.start(org.apache.nifi.controller.queue.clustered.partition.FlowFilePartitioner) > @bci=94, line=217 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.SocketLoadBalancedFlowFileQueue.setNodeIdentifiers(java.util.Set, > boolean) @bci=794, line=728 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.SocketLoadBalancedFlowFileQueue$ClusterEventListener.onNodeRemoved(org.apache.nifi.cluster.protocol.NodeIdentifier) > @bci=90, line=1082 (Compiled frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.lambda$onNodeRemoved$3(org.apache.nifi.cluster.protocol.NodeIdentifier, > org.apache.nifi.cluster.coordination.ClusterTopologyEventListener) @bci=2, > line=583 (Compiled frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator$$Lambda$1178.accept(java.lang.Object) > @bci=8 (Compiled frame) > - > java.util.concurrent.CopyOnWriteArrayList.forEach(java.util.function.Consumer) > @bci=38, line=891 (Compiled frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.onNodeRemoved(org.apache.nifi.cluster.protocol.NodeIdentifier) > @bci=10, line=583 (Interpreted frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.removeNodeConditionally(org.apache.nifi.cluster.protocol.NodeIdentifier, > org.apache.nifi.cluster.coordination.node.NodeConnectionStatus) @bci=29, > line=331 (Interpreted frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.replaceNodeStatus(org.apache.nifi.cluster.protocol.NodeIdentifier, > org.apache.nifi.cluster.coordination.node.NodeConnectionStatus, > org.apache.nifi.cluster.coordination.node.NodeConnectionStatus) @bci=89, > line=415 (Interpreted frame) > - > org.apache.nifi.cluster.coordination.node.NodeClusterCoordinator.resetNodeStatus(org.apache.nifi.cluster.coordination.node.NodeConnectionStatus, > long) @bci=44, line=380 (Interpreted frame) > - > org.apache.nifi.controller.cluster.ClusterProtocolHeartbeater.send(org.apache.nifi.cluster.protocol.message.HeartbeatMessage) > @bci=163, line=90 (Interpreted frame) > - org.apache.nifi.controller.FlowController$HeartbeatSendTask.run() > @bci=109, line=2692 (Interpreted frame) > - org.apache.nifi.engine.FlowEngine$2.run() @bci=4, line=110 (Compiled frame) > - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 > (Compiled frame) > - java.util.concurrent.FutureTask.runAndReset() @bci=47, line=308 (Compiled > frame) > - > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask) > @bci=1, line=180 (Compiled frame) > - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() > @bci=37, line=294 (Compiled frame) > - > java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) > @bci=95, line=1149 (Compiled frame) > - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 > (Interpreted frame) > - java.lang.Thread.run() @bci=11, line=748 (Compiled frame){code} > > - Holds NioAsyncLoadBalanceClient synchronized > - Needs SocketLoadBalancedFlowFileQueue.partitionReadLock > {code:java} > Thread 42352: (state = BLOCKED) > - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information > may be imprecise) > - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, > line=175 (Compiled frame) > - > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() > @bci=1, line=836 (Compiled frame) > - java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(int) > @bci=83, line=967 (Interpreted frame) > - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(int) > @bci=10, line=1283 (Compiled frame) > - java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock() @bci=5, > line=727 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.SocketLoadBalancedFlowFileQueue$1.isRebalanceOnFailure(org.apache.nifi.controller.queue.clustered.partition.FlowFilePartitioner) > @bci=7, line=375 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.partition.RemoteQueuePartition$1.isRebalanceOnFailure() > @bci=11, line=202 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.lambda$nodeDisconnected$0(org.apache.nifi.controller.queue.clustered.client.async.nio.RegisteredPartition) > @bci=4, line=304 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient$$Lambda$1144.test(java.lang.Object) > @bci=4 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.getReadyPartition(java.util.function.Predicate) > @bci=34, line=350 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClient.nodeDisconnected() > @bci=98, line=304 (Compiled frame) > - > org.apache.nifi.controller.queue.clustered.client.async.nio.NioAsyncLoadBalanceClientTask.run() > @bci=157, line=71 (Compiled frame) > - org.apache.nifi.engine.FlowEngine$2.run() @bci=4, line=110 (Interpreted > frame) > - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 > (Interpreted frame) > - java.util.concurrent.FutureTask.run() @bci=42, line=266 (Interpreted frame) > - > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask) > @bci=1, line=180 (Interpreted frame) > - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() > @bci=30, line=293 (Interpreted frame) > - > java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) > @bci=95, line=1149 (Interpreted frame) > - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 > (Interpreted frame) > - java.lang.Thread.run() @bci=11, line=748 (Interpreted frame) {code} -- This message was sent by Atlassian Jira (v8.3.4#803005)