janmee opened a new issue, #20148:
URL: https://github.com/apache/pulsar/issues/20148

   ### Search before asking
   
   - [X] I searched in the [issues](https://github.com/apache/pulsar/issues) 
and found nothing similar.
   
   
   ### Version
   
   2.11.0
   
   ### Minimal reproduce step
   
   1、create namespace and set bookie affinity group 
   2、create topic under that namespace
   3、broker will appear log "ZooKeeper session reconnection timeout. Notifying 
session is lost."
   
   
   ### What did you expect to see?
   
   create topic success
   
   ### What did you see instead?
   
   topic creating is block
   
   ### Anything else?
   
   
   `2023-04-19T21:53:11,365+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 4.166 seconds
   2023-04-19T21:53:11,366+0800 [metadata-store-zk-session-watcher-11-1] INFO  
org.apache.pulsar.broker.PulsarService - Received metadata service session 
event: ConnectionLost
   2023-04-19T21:53:11,367+0800 [metadata-store-zk-session-watcher-11-1] INFO  
org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl - Received 
MetadataStore session event: ConnectionLost
   2023-04-19T21:53:11,369+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 88,3  replyHeader:: 
88,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:11,701+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 3.83 seconds
   2023-04-19T21:53:11,702+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 89,3  replyHeader:: 
89,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:12,035+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 3.496 seconds
   2023-04-19T21:53:12,036+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 90,3  replyHeader:: 
90,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:12,368+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 3.163 seconds
   2023-04-19T21:53:12,369+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 91,3  replyHeader:: 
91,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:12,702+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 2.83 seconds
   2023-04-19T21:53:12,703+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 92,3  replyHeader:: 
92,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:13,035+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 2.496 seconds
   2023-04-19T21:53:13,036+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 93,3  replyHeader:: 
93,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:13,369+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 2.163 seconds
   2023-04-19T21:53:13,371+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 94,3  replyHeader:: 
94,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:13,702+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 1.829 seconds
   2023-04-19T21:53:13,703+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 95,3  replyHeader:: 
95,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:14,035+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 1.496 seconds
   2023-04-19T21:53:14,037+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 96,3  replyHeader:: 
96,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:14,369+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 1.162 seconds
   2023-04-19T21:53:14,370+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 97,3  replyHeader:: 
97,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:14,702+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 0.829 seconds
   2023-04-19T21:53:14,703+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 98,3  replyHeader:: 
98,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:15,036+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 0.495 seconds
   2023-04-19T21:53:15,037+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Reading reply session id: 0x24109391046d001e, 
packet:: clientPath:/ serverPath:/ finished:false header:: 99,3  replyHeader:: 
99,4294969298,0  request:: '/,F  response:: s{0,0,0,0,0,9,0,0,0,11,4294967580}
   2023-04-19T21:53:15,176+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Got notification session id: 
0x24109391046d001e
   2023-04-19T21:53:15,176+0800 [main-SendThread(11.238.117.36:2181)] DEBUG 
org.apache.zookeeper.ClientCnxn - Got WatchedEvent state:SyncConnected 
type:NodeDataChanged path:/loadbalance/brokers/11.238.117.36:8080 for session 
id 0x24109391046d001e
   2023-04-19T21:53:15,369+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.metadata.impl.ZKSessionWatcher - ZooKeeper client is 
disconnected. Waiting to reconnect, time remaining = 0.162 seconds
   2023-04-19T21:53:15,703+0800 [metadata-store-zk-session-watcher-11-1] INFO  
org.apache.pulsar.broker.PulsarService - Received metadata service session 
event: SessionLost
   2023-04-19T21:53:15,978+0800 [metadata-store-zk-session-watcher-11-1] WARN  
org.apache.pulsar.broker.PulsarService - The session with metadata service was 
lost. Shutting down.
   
   "main-EventThread" #19 daemon prio=5 os_prio=0 cpu=363.30ms elapsed=12.86s 
tid=0x00007fcd19a3b480 nid=0x3296 waiting on condition  [0x00007fcaf02e5000]
      java.lang.Thread.State: WAITING (parking)
           at jdk.internal.misc.Unsafe.park([email protected]/Native Method)
           - parking to wait for  <0x0000100015adbd50> (a 
java.util.concurrent.CompletableFuture$Signaller)
           at 
java.util.concurrent.locks.LockSupport.park([email protected]/LockSupport.java:211)
           at 
java.util.concurrent.CompletableFuture$Signaller.block([email protected]/CompletableFuture.java:1864)
           at 
java.util.concurrent.ForkJoinPool.unmanagedBlock([email protected]/ForkJoinPool.java:3463)
           at 
java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3434)
           at 
java.util.concurrent.CompletableFuture.waitingGet([email protected]/CompletableFuture.java:1898)
           at 
java.util.concurrent.CompletableFuture.join([email protected]/CompletableFuture.java:2117)
           at 
org.apache.pulsar.bookie.rackawareness.IsolatedBookieEnsemblePlacementPolicy.initialize(IsolatedBookieEnsemblePlacementPolicy.java:91)
           at 
org.apache.pulsar.bookie.rackawareness.IsolatedBookieEnsemblePlacementPolicy.initialize(IsolatedBookieEnsemblePlacementPolicy.java:51)
           at 
org.apache.bookkeeper.client.BookKeeper.initializeEnsemblePlacementPolicy(BookKeeper.java:583)
           at 
org.apache.bookkeeper.client.BookKeeper.<init>(BookKeeper.java:507)
           at 
org.apache.bookkeeper.client.BookKeeper$Builder.build(BookKeeper.java:307)
           at 
org.apache.pulsar.broker.BookKeeperClientFactoryImpl.create(BookKeeperClientFactoryImpl.java:83)
           at 
org.apache.pulsar.broker.ManagedLedgerClientFactory.lambda$initialize$0(ManagedLedgerClientFactory.java:97)
           at 
org.apache.pulsar.broker.ManagedLedgerClientFactory$$Lambda$715/0x0000000801192fb8.apply(Unknown
 Source)
           at 
java.util.concurrent.ConcurrentHashMap.computeIfAbsent([email protected]/ConcurrentHashMap.java:1708)
           - locked <0x0000100015aa9f60> (a 
java.util.concurrent.ConcurrentHashMap$ReservationNode)
           at 
org.apache.pulsar.broker.ManagedLedgerClientFactory.lambda$initialize$1(ManagedLedgerClientFactory.java:95)
           at 
org.apache.pulsar.broker.ManagedLedgerClientFactory$$Lambda$187/0x0000000800edc080.get(Unknown
 Source)
           at 
org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.lambda$asyncOpen$6(ManagedLedgerFactoryImpl.java:373)
           at 
org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl$$Lambda$709/0x0000000801186698.apply(Unknown
 Source)
           at 
java.util.concurrent.ConcurrentHashMap.computeIfAbsent([email protected]/ConcurrentHashMap.java:1708)
           - locked <0x0000100014f25a88> (a 
java.util.concurrent.ConcurrentHashMap$ReservationNode)
           at 
org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl.asyncOpen(ManagedLedgerFactoryImpl.java:369)
           at 
org.apache.pulsar.broker.service.BrokerService.lambda$createPersistentTopic$54(BrokerService.java:1429)
           at 
org.apache.pulsar.broker.service.BrokerService$$Lambda$707/0x000000080117f898.accept(Unknown
 Source)
           at 
java.util.concurrent.CompletableFuture.uniAcceptNow([email protected]/CompletableFuture.java:757)
           at 
java.util.concurrent.CompletableFuture.uniAcceptStage([email protected]/CompletableFuture.java:735)
           at 
java.util.concurrent.CompletableFuture.thenAccept([email protected]/CompletableFuture.java:2182)
           at 
org.apache.pulsar.broker.service.BrokerService.createPersistentTopic(BrokerService.java:1407)
           at 
org.apache.pulsar.broker.service.BrokerService.lambda$checkOwnershipAndCreatePersistentTopic$49(BrokerService.java:1375)
           at 
org.apache.pulsar.broker.service.BrokerService$$Lambda$698/0x000000080117a5c0.accept(Unknown
 Source)
           at 
java.util.concurrent.CompletableFuture.uniAcceptNow([email protected]/CompletableFuture.java:757)
           at 
java.util.concurrent.CompletableFuture.uniAcceptStage([email protected]/CompletableFuture.java:735)
           at 
java.util.concurrent.CompletableFuture.thenAccept([email protected]/CompletableFuture.java:2182)
           at 
org.apache.pulsar.broker.service.BrokerService.checkOwnershipAndCreatePersistentTopic(BrokerService.java:1377)
           at 
org.apache.pulsar.broker.service.BrokerService.lambda$loadOrCreatePersistentTopic$46(BrokerService.java:1350)
           at 
org.apache.pulsar.broker.service.BrokerService$$Lambda$708/0x00000008011af880.run(Unknown
 Source)
           at 
java.util.concurrent.CompletableFuture.uniRunNow([email protected]/CompletableFuture.java:819)
           at 
java.util.concurrent.CompletableFuture.uniRunStage([email protected]/CompletableFuture.java:803)
           at 
java.util.concurrent.CompletableFuture.thenRun([email protected]/CompletableFuture.java:2195)
           at 
org.apache.pulsar.broker.service.BrokerService.loadOrCreatePersistentTopic(BrokerService.java:1346)
           at 
org.apache.pulsar.broker.service.BrokerService.lambda$getTopic$23(BrokerService.java:991)
           at 
org.apache.pulsar.broker.service.BrokerService$$Lambda$701/0x00000008011ae8e8.apply(Unknown
 Source)
           at 
org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section.put(ConcurrentOpenHashMap.java:404)
           at 
org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap.computeIfAbsent(ConcurrentOpenHashMap.java:238)
           at 
org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:990)
           at 
org.apache.pulsar.broker.service.BrokerService.getTopic(BrokerService.java:952)
           at 
org.apache.pulsar.broker.service.BrokerService.getTopicIfExists(BrokerService.java:944)
           at 
org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.lambda$internalCreateNonPartitionedTopicAsync$49(PersistentTopicsBase.java:530)
           at 
org.apache.pulsar.broker.admin.impl.PersistentTopicsBase$$Lambda$616/0x000000080113d660.apply(Unknown
 Source)
           at 
java.util.concurrent.CompletableFuture$UniCompose.tryFire([email protected]/CompletableFuture.java:1150)
           at 
java.util.concurrent.CompletableFuture.postComplete([email protected]/CompletableFuture.java:510)
           at 
java.util.concurrent.CompletableFuture.complete([email protected]/CompletableFuture.java:2147)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore.handleGetResult(ZKMetadataStore.java:262)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$batchOperation$8(ZKMetadataStore.java:212)
           at 
org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$162/0x0000000800e8a448.processResult(Unknown
 Source)
           at 
org.apache.pulsar.metadata.impl.PulsarZooKeeperClient$3$1.processResult(PulsarZooKeeperClient.java:489)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:712)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:553)
   
      Locked ownable synchronizers:
           - None
   `
   broker log in ZKMetadataStore, we can see zk client has replied request, but 
ZKMetadataStore still disconnect. In the stack of zk EventThread, we know that 
EventThread got block in 
IsolatedBookieEnsemblePlacementPolicy.initialize(IsolatedBookieEnsemblePlacementPolicy.java:91)
 . 
   `
   public RackawareEnsemblePlacementPolicyImpl initialize(ClientConfiguration 
conf,
               Optional<DNSToSwitchMapping> optionalDnsResolver, 
HashedWheelTimer timer, FeatureProvider featureProvider,
               StatsLogger statsLogger, BookieAddressResolver 
bookieAddressResolver) {
           MetadataStore store;
           try {
               store = BookieRackAffinityMapping.createMetadataStore(conf);
           } catch (MetadataException e) {
               throw new RuntimeException(METADATA_STORE_INSTANCE + " failed 
initialized");
           }
           Set<String> primaryIsolationGroups = new HashSet<>();
           Set<String> secondaryIsolationGroups = new HashSet<>();
           if (conf.getProperty(ISOLATION_BOOKIE_GROUPS) != null) {
               String isolationGroupsString = ConfigurationStringUtil
                       .castToString(conf.getProperty(ISOLATION_BOOKIE_GROUPS));
               if (!isolationGroupsString.isEmpty()) {
                   for (String isolationGroup : 
isolationGroupsString.split(",")) {
                       primaryIsolationGroups.add(isolationGroup);
                   }
               }
               // Only add the bookieMappingCache if we have defined an 
isolation group
               bookieMappingCache = 
store.getMetadataCache(BookiesRackConfiguration.class);
               
bookieMappingCache.get(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH).join();
           }
           if (conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS) != null) {
               String secondaryIsolationGroupsString = ConfigurationStringUtil
                       
.castToString(conf.getProperty(SECONDARY_ISOLATION_BOOKIE_GROUPS));
               if (!secondaryIsolationGroupsString.isEmpty()) {
                   for (String isolationGroup : 
secondaryIsolationGroupsString.split(",")) {
                       secondaryIsolationGroups.add(isolationGroup);
                   }
               }
           }
           defaultIsolationGroups = ImmutablePair.of(primaryIsolationGroups, 
secondaryIsolationGroups);
           return super.initialize(conf, optionalDnsResolver, timer, 
featureProvider, statsLogger, bookieAddressResolver);
       }
   `
   in source code, we can see creating bookie client must wait until getting 
the node "/bookies" succeed from the cache. if  node "/bookies" is not in 
cache, ZKMetadataStore will get it from zk. However, creating bookie client is 
running in the zk EventThread, and waiting for getting node  "/bookies" 
complete, and the callback getting node "/bookies" from zk must run in 
EventThread。it will appear deadlock in this situation 
   
   
   ### Are you willing to submit a PR?
   
   - [ ] 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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to