You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@zookeeper.apache.org by "Michael Edgar (Jira)" <ji...@apache.org> on 2021/05/12 11:45:00 UTC

[jira] [Updated] (ZOOKEEPER-4293) Lock Contention in ClientCnxnSocketNetty (possible deadlock)

     [ https://issues.apache.org/jira/browse/ZOOKEEPER-4293?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Michael Edgar updated ZOOKEEPER-4293:
-------------------------------------
    Summary: Lock Contention in ClientCnxnSocketNetty (possible deadlock)  (was: Lock Contention in ClientCnxnSocketNetty)

> Lock Contention in ClientCnxnSocketNetty (possible deadlock)
> ------------------------------------------------------------
>
>                 Key: ZOOKEEPER-4293
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-4293
>             Project: ZooKeeper
>          Issue Type: Bug
>          Components: java client
>    Affects Versions: 3.5.8
>            Reporter: Michael Edgar
>            Priority: Critical
>         Attachments: strimzi-jstack-1620755249.log
>
>
> We have encountered a scenario where a `ClientCnxn#SendThread` has become blocked in `ClientCnxnSocketNetty#cleanup` in the call to `channel.close().syncUninterruptibly()`. As the `syncUninterruptibly` method blocks, processing in the `epollEventLoopGroup` thread for the same client is unable to obtain the lock in the listener associated with the client (line 151 in v3.5.8) and application threads using the `ZooKeeperAdmin` containing the `ClientCnxn` themselves become blocked on the same lock when attempting to close the client.
> The thread dump for the situation is attached, with the three interesting threads/stacks inlined below. Each occurrence lists three threads as it occurred to three clients simultaneously. 
> h2. SendThread
> * vert.x-eventloop-thread-0-SendThread(foo-0xtq07v5viyjxv8-zookeeper-0.foo-0xtq07v5viyjxv8-zookeeper-nodes.foo-0xtq07v5viyjxv8.svc:2181) awaiting notification , holding [ 0x00000000f6b5fdd0 ]
> * vert.x-eventloop-thread-0-SendThread(foo-lgr9apuzr7ngznmd-zookeeper-2.foo-lgr9apuzr7ngznmd-zookeeper-nodes.foo-lgr9apuzr7ngznmd.svc:2181) awaiting notification , holding [ 0x00000000f6b8e5c0 ]
> * vert.x-eventloop-thread-0-SendThread(foo-tukt40lcdodsux6n-zookeeper-1.foo-tukt40lcdodsux6n-zookeeper-nodes.foo-tukt40lcdodsux6n.svc:2181) awaiting notification , holding [ 0x00000000f6b9d5a0 ] {code}at java.lang.Object.wait(java.base@11.0.11/Native Method)
> at java.lang.Object.wait(java.base@11.0.11/Object.java:328)
> at io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:275)
> at io.netty.channel.DefaultChannelPromise.awaitUninterruptibly(DefaultChannelPromise.java:137)
> at io.netty.channel.DefaultChannelPromise.awaitUninterruptibly(DefaultChannelPromise.java:30)
> at io.netty.util.concurrent.DefaultPromise.syncUninterruptibly(DefaultPromise.java:411)
> at io.netty.channel.DefaultChannelPromise.syncUninterruptibly(DefaultChannelPromise.java:125)
> at io.netty.channel.DefaultChannelPromise.syncUninterruptibly(DefaultChannelPromise.java:30)
> at org.apache.zookeeper.ClientCnxnSocketNetty.cleanup(ClientCnxnSocketNetty.java:212)
> at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1338)
> at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1276)
> at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1254){code}
> h2. epollEventLoopGroup
> * epollEventLoopGroup-11-1 awaiting notification on [ 0x00000000f6b8e5c0 ]
> * epollEventLoopGroup-19-1 awaiting notification on [ 0x00000000f6b9d5a0 ]
> * epollEventLoopGroup-2-1 awaiting notification on [ 0x00000000f6b5fdd0 ] {code}at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method)
> at java.util.concurrent.locks.LockSupport.park(java.base@11.0.11/LockSupport.java:194)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(java.base@11.0.11/AbstractQueuedSynchronizer.java:885)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.base@11.0.11/AbstractQueuedSynchronizer.java:917)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(java.base@11.0.11/AbstractQueuedSynchronizer.java:1240)
> at java.util.concurrent.locks.ReentrantLock.lock(java.base@11.0.11/ReentrantLock.java:267)
> at org.apache.zookeeper.ClientCnxnSocketNetty$1.operationComplete(ClientCnxnSocketNetty.java:151)
> at org.apache.zookeeper.ClientCnxnSocketNetty$1.operationComplete(ClientCnxnSocketNetty.java:146)
> at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:578)
> at io.netty.util.concurrent.DefaultPromise.notifyListeners0(DefaultPromise.java:571)
> at io.netty.util.concurrent.DefaultPromise.notifyListenersNow(DefaultPromise.java:550)
> at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:491)
> at io.netty.util.concurrent.DefaultPromise.setValue0(DefaultPromise.java:616)
> at io.netty.util.concurrent.DefaultPromise.setSuccess0(DefaultPromise.java:605)
> at io.netty.util.concurrent.DefaultPromise.trySuccess(DefaultPromise.java:104)
> at io.netty.channel.DefaultChannelPromise.trySuccess(DefaultChannelPromise.java:84)
> at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.fulfillConnectPromise(AbstractEpollChannel.java:653)
> at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.finishConnect(AbstractEpollChannel.java:691)
> at io.netty.channel.epoll.AbstractEpollChannel$AbstractEpollUnsafe.epollOutReady(AbstractEpollChannel.java:567)
> at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:470)
> at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:378)
> at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)
> at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)
> at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.lang.Thread.run(java.base@11.0.11/Thread.java:829){code}
> h2. vert.x-worker-thread (application thread)
> * vert.x-worker-thread-16 awaiting notification on [ 0x00000000f6b8e5c0 ] , holding [ 0x00000000f6b8d148 0x00000000f6b76870 ]
> * vert.x-worker-thread-19 awaiting notification on [ 0x00000000f6b9d5a0 ] , holding [    0x00000000f6b9b4c8 0x00000000f6b76df8 ]
> * vert.x-worker-thread-4 awaiting notification on [ 0x00000000f6b5fdd0 ] , holding [    0x00000000f6b5ab08 0x00000000f6b43a98 ] {code}at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method)
> at java.util.concurrent.locks.LockSupport.park(java.base@11.0.11/LockSupport.java:194)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(java.base@11.0.11/AbstractQueuedSynchronizer.java:885)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.base@11.0.11/AbstractQueuedSynchronizer.java:917)
> at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(java.base@11.0.11/AbstractQueuedSynchronizer.java:1240)
> at java.util.concurrent.locks.ReentrantLock.lock(java.base@11.0.11/ReentrantLock.java:267)
> at org.apache.zookeeper.ClientCnxnSocketNetty.cleanup(ClientCnxnSocketNetty.java:205)
> at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1338)
> at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1276)
> at org.apache.zookeeper.ClientCnxn$SendThread.access$2800(ClientCnxn.java:805)
> at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1534)
> at org.apache.zookeeper.ClientCnxn.submitRequest(ClientCnxn.java:1512)
> at org.apache.zookeeper.ClientCnxn.close(ClientCnxn.java:1481)
> at org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:1415)
> at org.apache.zookeeper.ZooKeeper.close(ZooKeeper.java:1437)
> at io.strimzi.operator.cluster.operator.resource.ZookeeperScaler.lambda$closeConnection$10(ZookeeperScaler.java:244)
> at io.strimzi.operator.cluster.operator.resource.ZookeeperScaler$$Lambda$501/0x0000000840629840.handle(Unknown Source)
> at io.vertx.core.impl.ContextImpl.lambda$null$0(ContextImpl.java:179)
> at io.vertx.core.impl.ContextImpl$$Lambda$208/0x00000008402ea440.handle(Unknown Source)
> at io.vertx.core.impl.AbstractContext.dispatch(AbstractContext.java:96)
> at io.vertx.core.impl.ContextImpl.lambda$executeBlocking$1(ContextImpl.java:177)
> at io.vertx.core.impl.ContextImpl$$Lambda$205/0x00000008402e9040.run(Unknown Source)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.11/ThreadPoolExecutor.java:1128)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.11/ThreadPoolExecutor.java:628)
> at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> at java.lang.Thread.run(java.base@11.0.11/Thread.java:829){code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)