You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by "janmee (via GitHub)" <gi...@apache.org> on 2023/04/20 03:47:12 UTC

[GitHub] [pulsar] janmee opened a new issue, #20148: [Bug] ZKMetadataStore run into deadlock when creating topic using IsolatedBookieEnsemblePlacement in missing cache

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(java.base@17.0.5/Native Method)
           - parking to wait for  <0x0000100015adbd50> (a java.util.concurrent.CompletableFuture$Signaller)
           at java.util.concurrent.locks.LockSupport.park(java.base@17.0.5/LockSupport.java:211)
           at java.util.concurrent.CompletableFuture$Signaller.block(java.base@17.0.5/CompletableFuture.java:1864)
           at java.util.concurrent.ForkJoinPool.unmanagedBlock(java.base@17.0.5/ForkJoinPool.java:3463)
           at java.util.concurrent.ForkJoinPool.managedBlock(java.base@17.0.5/ForkJoinPool.java:3434)
           at java.util.concurrent.CompletableFuture.waitingGet(java.base@17.0.5/CompletableFuture.java:1898)
           at java.util.concurrent.CompletableFuture.join(java.base@17.0.5/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(java.base@17.0.5/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(java.base@17.0.5/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(java.base@17.0.5/CompletableFuture.java:757)
           at java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@17.0.5/CompletableFuture.java:735)
           at java.util.concurrent.CompletableFuture.thenAccept(java.base@17.0.5/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(java.base@17.0.5/CompletableFuture.java:757)
           at java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@17.0.5/CompletableFuture.java:735)
           at java.util.concurrent.CompletableFuture.thenAccept(java.base@17.0.5/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(java.base@17.0.5/CompletableFuture.java:819)
           at java.util.concurrent.CompletableFuture.uniRunStage(java.base@17.0.5/CompletableFuture.java:803)
           at java.util.concurrent.CompletableFuture.thenRun(java.base@17.0.5/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(java.base@17.0.5/CompletableFuture.java:1150)
           at java.util.concurrent.CompletableFuture.postComplete(java.base@17.0.5/CompletableFuture.java:510)
           at java.util.concurrent.CompletableFuture.complete(java.base@17.0.5/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: commits-unsubscribe@pulsar.apache.org.apache.org

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


Re: [I] [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache [pulsar]

Posted by "Technoboy- (via GitHub)" <gi...@apache.org>.
Technoboy- closed issue #20148: [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache
URL: https://github.com/apache/pulsar/issues/20148


-- 
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-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] poorbarcode commented on issue #20148: [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache

Posted by "poorbarcode (via GitHub)" <gi...@apache.org>.
poorbarcode commented on issue #20148:
URL: https://github.com/apache/pulsar/issues/20148#issuecomment-1593290151

   > https://github.com/apache/pulsar/pull/20303 handles a related issue that is possibly solving this one also.
   
   No, the PR https://github.com/apache/pulsar/pull/20303 is not helpful for this issue. I'll take note of the issue and will try to fix it later.


-- 
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-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] tisonkun commented on issue #20148: [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache

Posted by "tisonkun (via GitHub)" <gi...@apache.org>.
tisonkun commented on issue #20148:
URL: https://github.com/apache/pulsar/issues/20148#issuecomment-1593144676

   https://github.com/apache/pulsar/pull/20303 handles a related issue that is possibly solve this one also.
   
   cc @poorbarcode 


-- 
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-unsubscribe@pulsar.apache.org

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


Re: [I] [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache [pulsar]

Posted by "Technoboy- (via GitHub)" <gi...@apache.org>.
Technoboy- commented on issue #20148:
URL: https://github.com/apache/pulsar/issues/20148#issuecomment-1868989315

   https://github.com/apache/pulsar/pull/21096 has fixed this issue


-- 
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-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on issue #20148: [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #20148:
URL: https://github.com/apache/pulsar/issues/20148#issuecomment-1560351669

   The issue had no activity for 30 days, mark with Stale label.


-- 
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-unsubscribe@pulsar.apache.org

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


[GitHub] [pulsar] github-actions[bot] commented on issue #20148: [Bug] Broker gets block while creating topic using IsolatedBookieEnsemblePlacement in missing cache

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on issue #20148:
URL: https://github.com/apache/pulsar/issues/20148#issuecomment-1636951774

   The issue had no activity for 30 days, mark with Stale label.


-- 
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-unsubscribe@pulsar.apache.org

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