tiny-rain opened a new issue, #20105:
URL: https://github.com/apache/pulsar/issues/20105

   ### Search before asking
   
   - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) 
and found nothing similar.
   
   
   ### Version
   
   Pulsar version: 2.11
   OS: MacOs Monterey
   
   
   ### Minimal reproduce step
   
   1. make zk connection lost of zkc in ZKMetadataStore
   2. wait connection check thread in ZkSessionWather begin to run.
   
   ### What did you expect to see?
   
   - **thread1:** In the session watch callback of ZKMetadataStore, it will 
firstly lock the pending request queue in ClientCnxn class,and then call 
zkSessionWather.process, which need acquire lock of zkSessionWatcher.
   
   - **Thread2:** While the schedule connection check thread in ZkSessionWather 
begin running, execute zookeeper.exists in 
ZkSessionWather.checkConnectionStatus(), in this way, the lock of 
zkSessionWatcher is hold by connectionStatusCheck thread, and waiting lock of 
pendingrequest queue in ClientCnxn, which was already holed by the session 
wather callback thread.
   
   Risk: all the metadata request are blocked.
    
   
   ### What did you see instead?
   
   
   Java stack information for the threads listed above:
   ===================================================
   "metadata-store-40-1":
           at 
org.apache.pulsar.metadata.impl.ZKSessionWatcher.process(ZKSessionWatcher.java:120)
           - waiting to lock <0x000000070096de18> (a 
org.apache.pulsar.metadata.impl.ZKSessionWatcher)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$new$0(ZKMetadataStore.java:101)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$818/0x0000000801684680.run(Unknown
 Source)
           at 
java.util.concurrent.Executors$RunnableAdapter.call(java.base@17.0.6/Executors.java:539)
           at 
java.util.concurrent.FutureTask.run$$$capture(java.base@17.0.6/FutureTask.java:264)
           at 
java.util.concurrent.FutureTask.run(java.base@17.0.6/FutureTask.java)
           at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@17.0.6/ScheduledThreadPoolExecutor.java:304)
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.6/ThreadPoolExecutor.java:1136)
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.6/ThreadPoolExecutor.java:635)
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
           at java.lang.Thread.run(java.base@17.0.6/Thread.java:833)
   "metadata-store-zk-session-watcher-41-1":
           at 
org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:527)
           - waiting to lock <0x0000000706564568> (a 
java.util.concurrent.LinkedBlockingQueue)
           at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:771)
           at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:808)
           at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1676)
           - locked <0x0000000700d27530> (a 
org.apache.zookeeper.ZooKeeper$States)
           at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1645)
           at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1919)
           at org.apache.zookeeper.ZooKeeper.exists(ZooKeeper.java:1930)
           at 
org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$16.zkRun(PulsarZooKeeperClient.java:926)
           at 
org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$ZkRetryRunnable.run(PulsarZooKeeperClient.java:390)
           at 
org.apache.pulsar.metadata.impl.PulsarZooKeeperClient.exists(PulsarZooKeeperClient.java:936)
           at 
org.apache.pulsar.metadata.impl.ZKSessionWatcher.checkConnectionStatus(ZKSessionWatcher.java:86)
           - locked <0x000000070096de18> (a 
org.apache.pulsar.metadata.impl.ZKSessionWatcher)
           at 
org.apache.pulsar.metadata.impl.ZKSessionWatcher$$Lambda$328/0x00000008013d3470.run(Unknown
 Source)
           at 
org.apache.pulsar.common.util.Runnables$CatchingAndLoggingRunnable.run(Runnables.java:54)
           at 
java.util.concurrent.Executors$RunnableAdapter.call(java.base@17.0.6/Executors.java:539)
           at 
java.util.concurrent.FutureTask.runAndReset$$$capture(java.base@17.0.6/FutureTask.java:305)
           at 
java.util.concurrent.FutureTask.runAndReset(java.base@17.0.6/FutureTask.java)
           at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(java.base@17.0.6/ScheduledThreadPoolExecutor.java:305)
           at 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@17.0.6/ThreadPoolExecutor.java:1136)
           at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@17.0.6/ThreadPoolExecutor.java:635)
           at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
           at java.lang.Thread.run(java.base@17.0.6/Thread.java:833)
   "ZKC-connect-executor-0-SendThread(localhost:2181)":
           at 
org.apache.pulsar.metadata.impl.ZKSessionWatcher.setSessionInvalid(ZKSessionWatcher.java:125)
           - waiting to lock <0x000000070096de18> (a 
org.apache.pulsar.metadata.impl.ZKSessionWatcher)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$823/0x00000008016866c8.accept(Unknown
 Source)
           at java.util.Optional.ifPresent(java.base@17.0.6/Optional.java:178)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$receivedSessionEvent$3(ZKMetadataStore.java:146)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$821/0x0000000801685dd8.processResult(Unknown
 Source)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:726)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:531)
           - locked <0x0000000706564568> (a 
java.util.concurrent.LinkedBlockingQueue)
           at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:771)
           at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:808)
           at org.apache.zookeeper.ClientCnxn.access$2500(ClientCnxn.java:106)
           at 
org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1399)
           at 
org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1327)
           at 
org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1301)
   
   Found 1 deadlock.
   
   
   ### Anything else?
   
   _No response_
   
   ### Are you willing to submit a PR?
   
   - [X] I'm willing to submit a PR!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@pulsar.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to