-
Notifications
You must be signed in to change notification settings - Fork 3.7k
Description
Describe the bug
(
pulsar/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java
Line 594 in 2b3e8ae
| public synchronized void setupTopicPublishRateLimiterMonitor() { |
This can block forever, causing metadata operations to deadlock.
"metadata-store-6-1" #61 prio=5 os_prio=0 cpu=139825.70ms elapsed=81027.01s tid=0x00007f1598001000 nid=0x8b waiting for monitor entry [0x00007f15a7805000]
java.lang.Thread.State: BLOCKED (on object monitor)
at org.apache.pulsar.broker.service.BrokerService.setupTopicPublishRateLimiterMonitor(BrokerService.java:562)
- waiting to lock <0x00000003018b1c30> (a org.apache.pulsar.broker.service.BrokerService)
at org.apache.pulsar.broker.service.AbstractTopic.updatePublishDispatcher(AbstractTopic.java:920)
at org.apache.pulsar.broker.service.AbstractTopic.updatePublishDispatcher(AbstractTopic.java:807)
at org.apache.pulsar.broker.service.AbstractTopic.updateMaxPublishRate(AbstractTopic.java:770)
at org.apache.pulsar.broker.service.persistent.PersistentTopic.onPoliciesUpdate(PersistentTopic.java:2424)
at org.apache.pulsar.broker.service.BrokerService.lambda$null$81(BrokerService.java:1807)
at org.apache.pulsar.broker.service.BrokerService$$Lambda$1326/0x00000008408bc440.accept(Unknown Source)
at java.util.Optional.ifPresent(java.base@11.0.13/Optional.java:183)
at org.apache.pulsar.broker.service.BrokerService.lambda$null$82(BrokerService.java:1807)
at org.apache.pulsar.broker.service.BrokerService$$Lambda$1325/0x00000008408bc040.accept(Unknown Source)
at java.util.concurrent.CompletableFuture.uniAcceptNow(java.base@11.0.13/CompletableFuture.java:753)
at java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@11.0.13/CompletableFuture.java:731)
at java.util.concurrent.CompletableFuture.thenAccept(java.base@11.0.13/CompletableFuture.java:2108)
at org.apache.pulsar.broker.service.BrokerService.lambda$null$83(BrokerService.java:1802)
at org.apache.pulsar.broker.service.BrokerService$$Lambda$678/0x0000000840655840.accept(Unknown Source)
at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap$Section.forEach(ConcurrentOpenHashMap.java:387)
at org.apache.pulsar.common.util.collections.ConcurrentOpenHashMap.forEach(ConcurrentOpenHashMap.java:159)
at org.apache.pulsar.broker.service.BrokerService.lambda$handlePoliciesUpdates$84(BrokerService.java:1798)
at org.apache.pulsar.broker.service.BrokerService$$Lambda$677/0x0000000840655440.accept(Unknown Source)
at java.util.concurrent.CompletableFuture$UniAccept.tryFire(java.base@11.0.13/CompletableFuture.java:714)
at java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.13/CompletableFuture.java:506)
at java.util.concurrent.CompletableFuture.complete(java.base@11.0.13/CompletableFuture.java:2073)
at org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$null$7(ZKMetadataStore.java:139)
at org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$226/0x000000084034fc40.run(Unknown Source)
at org.apache.pulsar.metadata.impl.AbstractMetadataStore$TaskWrapper.run(AbstractMetadataStore.java:99)
at java.util.concurrent.Executors$RunnableAdapter.call(java.base@11.0.13/Executors.java:515)
at java.util.concurrent.FutureTask.run(java.base@11.0.13/FutureTask.java:264)
at java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.13/ThreadPoolExecutor.java:1128)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.13/ThreadPoolExecutor.java:628)
at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.lang.Thread.run(java.base@11.0.13/Thread.java:829)
To Reproduce
No reliable way to reproduce the problem, though we have seen this in production deployment.
Expected behavior
This method should not block.