You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/09/26 12:56:55 UTC

[GitHub] [pulsar] wuzhanpeng opened a new pull request #12202: Fix getting stuck when adding producers

wuzhanpeng opened a new pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202


   ### Motivation
   
   In our production environment, when the broker receives a large number of `PRODUCER` requests in a short period of time, we have observed that the broker will have a loop waiting problem when handling these requests, which will cause the broker to get stuck in severe cases and cause a huge amonut of other requests to time out.
   
   To simplify the description of the problem, we assume that multiple producers initiate `PRODUCER` requests to a topic at the same time. As we can see in `AbstractTopic#addProducer`, we can simplify the process so that the process of `addProducer` in each `PRODUCER` request is broken down into: 
   
   1. acquire the lock in thread#1
   2. load data from zk in thread#2 with timeout (i.e. `AbstractTopic#isProducersExceeded` in internal adding producer)
   3. return back to thread#1 to release the lock
   
   It should be noted that these 3 processes are serial.
   
   Assuming that the core size of the thread pool(actually is `ForkJoinPool.commonPool()`) that processes the above threads is only 1, and only one thread can successfully obtain the lock(`AbstractTopic#lock`) in the simultaneous `PRODUCER` requests, the remaining threads must be queued in the submission queue of the thread pool. Unfortunately, there is a high probability that thread#2 will be put into the queue waiting for scheduling. In this situation, the thread#1 that acquired the lock cannot complete because it needs to wait for the thread#2, and the other threads that have not acquired the lock need to acquire the lock first. This process cannot continue until the thread#2 times out and throws an exception.
   
   For jstack result, we can easily see
   
   ```
   "ForkJoinPool.commonPool-worker-110" #482 daemon prio=5 os_prio=0 tid=0x00007fd714021000 nid=0x61a3 waiting on condition  [0x00007fd562772000]
      java.lang.Thread.State: TIMED_WAITING (parking)
           at jdk.internal.misc.Unsafe.park(java.base@10.0.2/Native Method)
           - parking to wait for  <0x00000006284caad0> (a java.util.concurrent.CompletableFuture$Signaller)
           at java.util.concurrent.locks.LockSupport.parkNanos(java.base@10.0.2/LockSupport.java:234)
           at java.util.concurrent.CompletableFuture$Signaller.block(java.base@10.0.2/CompletableFuture.java:1798)
           at java.util.concurrent.ForkJoinPool.managedBlock(java.base@10.0.2/ForkJoinPool.java:3146)
           at java.util.concurrent.CompletableFuture.timedGet(java.base@10.0.2/CompletableFuture.java:1868)
           at java.util.concurrent.CompletableFuture.get(java.base@10.0.2/CompletableFuture.java:2021)
           at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
           at org.apache.pulsar.broker.service.AbstractTopic.isProducersExceeded(AbstractTopic.java:156)
           at org.apache.pulsar.broker.service.AbstractTopic.internalAddProducer(AbstractTopic.java:629)
           at org.apache.pulsar.broker.service.AbstractTopic.lambda$addProducer$8(AbstractTopic.java:405)
           at org.apache.pulsar.broker.service.AbstractTopic$$Lambda$1433/1422007940.accept(Unknown Source)
           at java.util.concurrent.CompletableFuture.uniAcceptNow(java.base@10.0.2/CompletableFuture.java:753)
           at java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@10.0.2/CompletableFuture.java:731)
           at java.util.concurrent.CompletableFuture.thenAccept(java.base@10.0.2/CompletableFuture.java:2108)
           at org.apache.pulsar.broker.service.AbstractTopic.addProducer(AbstractTopic.java:392)
           at org.apache.pulsar.broker.service.persistent.PersistentTopic.addProducer(PersistentTopic.java:540)
           at org.apache.pulsar.broker.service.ServerCnx.lambda$null$22(ServerCnx.java:1233)
           at org.apache.pulsar.broker.service.ServerCnx$$Lambda$1428/932296811.accept(Unknown Source)
           at java.util.concurrent.CompletableFuture$UniAccept.tryFire(java.base@10.0.2/CompletableFuture.java:714)
           at java.util.concurrent.CompletableFuture.postComplete(java.base@10.0.2/CompletableFuture.java:506)
           at java.util.concurrent.CompletableFuture.complete(java.base@10.0.2/CompletableFuture.java:2073)
           at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.lambda$null$6(BookkeeperSchemaStorage.java:217)
           at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$$Lambda$1421/1611023719.apply(Unknown Source)
           at java.util.concurrent.CompletableFuture.uniHandle(java.base@10.0.2/CompletableFuture.java:930)
           at java.util.concurrent.CompletableFuture$UniHandle.tryFire(java.base@10.0.2/CompletableFuture.java:907)
           at java.util.concurrent.CompletableFuture$Completion.exec(java.base@10.0.2/CompletableFuture.java:479)
           at java.util.concurrent.ForkJoinTask.doExec(java.base@10.0.2/ForkJoinTask.java:290)
           at java.util.concurrent.ForkJoinPool.runWorker(java.base@10.0.2/ForkJoinPool.java:1603)
           at java.util.concurrent.ForkJoinWorkerThread.run(java.base@10.0.2/ForkJoinWorkerThread.java:177)
   
      Locked ownable synchronizers:
           - <0x0000000624e2e9a0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
   ```
   
   To make matters worse, in the default configuration, both zk timeout and client operation timeout are 30 seconds. This will cause each retry request to end with a timeout and iteratively.
   
   ### Modifications
   
   This problem is so hidden that it is very difficult to detect, and even we still have no way to reproduce it in the test environment. However, in our production environment, this problem is more likely to occur if the bundle re-load or broker restart operation is triggered frequently(This phenomenon may be more obvious in our production scenarios. Each of our independent topics may have thousands of producers). Once the problem occurs in the cluster, there will be a lot of operation timeout exceptions.
   
   Below we give a solution to reduce the use of locks, because we think that for the conventional production model, it is sufficient to use read locks when adding producers in `Shared` mode.


-- 
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] Anonymitaet commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
Anonymitaet commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-930941229


   Thanks for your contribution. For this PR, do we need to update docs?
   
   (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks) 


-- 
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] hangc0276 edited a comment on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
hangc0276 edited a comment on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928788414


   I doubt the write lock maybe not the root cause of this issue. 
   
   When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked. 
   
   So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`. You can also the stack about ZooKeeperDataCache.


-- 
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] wuzhanpeng edited a comment on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng edited a comment on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-929973940


   > I doubt the write lock maybe not the root cause of this issue.
   > 
   > When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked.
   > 
   > So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`. You can also the stack about ZooKeeperDataCache.
   
   Thank you for your reminder~ 
   
   We also checked why the caching strategy of ns policy did not take effect. The actual situation is that when the broker gets into a loop waiting problem, every time the `ZooKeeperDataCache#get` times out, it will invalidate the z-path by the way. In this way, the next time you get the ns strategy, you still have to get data from zk. Therefore, once a problem occurs, it is difficult to cache successfully and then get out of the predicament.
   
   There are many ways to break the deadlock condition in this scenario. However, IMHO reducing the use of locks may be a more thorough solution. After all, if the producers of shared mode accounts for most of the topics, the existence of this lock itself is also reducing the overall performance. In addition, the logic involved in the cache layer is extensive, and avoiding modifying the current cache design may be a more secure solution.


-- 
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] wuzhanpeng commented on a change in pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on a change in pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#discussion_r717176613



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
##########
@@ -404,7 +406,9 @@ public String getReplicatorPrefix() {
                 .thenCompose(__ ->
                         incrementTopicEpochIfNeeded(producer, producerQueuedFuture))
                 .thenCompose(producerEpoch -> {
-                    lock.writeLock().lock();
+                    Lock producerLock = producer.getAccessMode() == ProducerAccessMode.Shared
+                            ? lock.readLock() : lock.writeLock();
+                    producerLock.lock();

Review comment:
       Thanks for your reply! Let me take the liberty to ask, what does `counter` refer to in `internalAddProducer`?




-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-930682783


   > > every time the `ZooKeeperDataCache#get` times out
   > 
   > @wuzhanpeng Would you address the reason of `every time the ZooKeeperDataCache#get times out`?
   
   As I described in motivation, when the core pool is occupied by all those threads waiting for the lock(identified as thread#1 above), the thread(thread#2) that loads ns policy from zk can only wait in the submission queue until it times out.


-- 
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] wuzhanpeng edited a comment on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng edited a comment on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-929973940


   > I doubt the write lock maybe not the root cause of this issue.
   > 
   > When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked.
   > 
   > So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`. You can also the stack about ZooKeeperDataCache.
   
   Thank you for your reminder~ 
   
   We also checked why the caching strategy of ns policy did not take effect. The actual situation is that when the broker gets into a loop waiting problem, every time the `ZooKeeperDataCache#get` times out, it will invalidate the z-path by the way. In this way, the next time you get the ns strategy, you still have to get data from zk. Therefore, once a problem occurs, it is difficult to cache successfully and then get out of the predicament.
   
   There are many ways to break the deadlock condition in this scenario. However, IMHO reducing the use of locks may be a more thorough solution. After all, if the producers of shared mode accounts for most of the topics, the existence of this lock itself is also reducing the overall performance. In addition, the logic involved in the cache layer is extensive, and avoiding modifying the current cache design may be a more secure solution.


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-938275806


   > Thanks for your contribution. For this PR, do we need to update docs?
   > 
   > (The [PR template contains info about doc](https://github.com/apache/pulsar/blob/master/.github/PULL_REQUEST_TEMPLATE.md#documentation), which helps others know more about the changes. Can you provide doc-related info in this and future PR descriptions? Thanks)
   
   @Anonymitaet Thanks for your reminder. No need to update the documentation.


-- 
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] michaeljmarshall commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-1036548745


   Removing the `release/2.8.3` label since this will miss the release. 


-- 
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 pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-1066260028


   The pr 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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-929951024






-- 
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] codelipenghui commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-927867333


   @wuzhanpeng Do you have the complete stack information? Not sure if there is dead lock in the metadata cache.


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-929973940


   > I doubt the write lock maybe not the root cause of this issue.
   > 
   > When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked.
   > 
   > So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`. You can also the stack about ZooKeeperDataCache.
   
   Thank you for your reminder~ 
   
   We also checked why the caching strategy of ns policy did not take effect. The actual situation is that when the broker gets into a loop waiting problem, every time the `ZooKeeperDataCache#get` times out, it will invalidate the z-path by the way. In this way, the next time you get the ns strategy, you still have to get data from zk. Therefore, once the problem is present, it is difficult to succeed in the first cached and then get out of the trouble.
   
   There are many ways to break the deadlock condition in this scenario. However, IMHO reducing the use of locks may be a more thorough solution. After all, if the producers of shared mode accounts for most of the topics, the existence of this lock itself is also reducing the overall performance. In addition, the logic involved in the cache layer is extensive, and avoiding modifying the current cache design may be a more secure solution.


-- 
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] codelipenghui edited a comment on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
codelipenghui edited a comment on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-927867333


   @wuzhanpeng Do you have the complete stack information? Not sure if there is dead lock in the metadata cache. Have you tried to restart the broker to see if the problem can be resolved.


-- 
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] hangc0276 commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-930655249


   > every time the `ZooKeeperDataCache#get` times out
   
   @wuzhanpeng Would you address the reason of `every time the ZooKeeperDataCache#get times out`?


-- 
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] hangc0276 edited a comment on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
hangc0276 edited a comment on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928788414


   I doubt the write lock maybe not the root cause of this issue. 
   
   When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked. 
   
   So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`. You can also the stack about ZooKeeperDataCache.


-- 
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] 315157973 commented on a change in pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
315157973 commented on a change in pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#discussion_r716627314



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
##########
@@ -404,7 +406,9 @@ public String getReplicatorPrefix() {
                 .thenCompose(__ ->
                         incrementTopicEpochIfNeeded(producer, producerQueuedFuture))
                 .thenCompose(producerEpoch -> {
-                    lock.writeLock().lock();
+                    Lock producerLock = producer.getAccessMode() == ProducerAccessMode.Shared
+                            ? lock.readLock() : lock.writeLock();
+                    producerLock.lock();

Review comment:
       The counter in internalAddProducer() is not thread-safe, it will be a problem if you use the read lock




-- 
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] codelipenghui commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
codelipenghui commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-1015036740


   @wuzhanpeng After taking a look at the complete stack, looks the issue is related to checking topic policies 
   
   ```
   "ForkJoinPool.commonPool-worker-110" #482 daemon prio=5 os_prio=0 tid=0x00007fd714021000 nid=0x61a3 waiting on condition  [0x00007fd562772000]
      java.lang.Thread.State: TIMED_WAITING (parking)
   	at jdk.internal.misc.Unsafe.park(java.base@10.0.2/Native Method)
   	- parking to wait for  <0x00000006284caad0> (a java.util.concurrent.CompletableFuture$Signaller)
   	at java.util.concurrent.locks.LockSupport.parkNanos(java.base@10.0.2/LockSupport.java:234)
   	at java.util.concurrent.CompletableFuture$Signaller.block(java.base@10.0.2/CompletableFuture.java:1798)
   	at java.util.concurrent.ForkJoinPool.managedBlock(java.base@10.0.2/ForkJoinPool.java:3146)
   	at java.util.concurrent.CompletableFuture.timedGet(java.base@10.0.2/CompletableFuture.java:1868)
   	at java.util.concurrent.CompletableFuture.get(java.base@10.0.2/CompletableFuture.java:2021)
   	at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
   	at org.apache.pulsar.broker.service.AbstractTopic.isProducersExceeded(AbstractTopic.java:156)
   	at org.apache.pulsar.broker.service.AbstractTopic.internalAddProducer(AbstractTopic.java:629)
   	at org.apache.pulsar.broker.service.AbstractTopic.lambda$addProducer$8(AbstractTopic.java:405)
   	at org.apache.pulsar.broker.service.AbstractTopic$$Lambda$1433/1422007940.accept(Unknown Source)
   	at java.util.concurrent.CompletableFuture.uniAcceptNow(java.base@10.0.2/CompletableFuture.java:753)
   	at java.util.concurrent.CompletableFuture.uniAcceptStage(java.base@10.0.2/CompletableFuture.java:731)
   	at java.util.concurrent.CompletableFuture.thenAccept(java.base@10.0.2/CompletableFuture.java:2108)
   	at org.apache.pulsar.broker.service.AbstractTopic.addProducer(AbstractTopic.java:392)
   	at org.apache.pulsar.broker.service.persistent.PersistentTopic.addProducer(PersistentTopic.java:540)
   	at org.apache.pulsar.broker.service.ServerCnx.lambda$null$22(ServerCnx.java:1233)
   	at org.apache.pulsar.broker.service.ServerCnx$$Lambda$1428/932296811.accept(Unknown Source)
   	at java.util.concurrent.CompletableFuture$UniAccept.tryFire(java.base@10.0.2/CompletableFuture.java:714)
   	at java.util.concurrent.CompletableFuture.postComplete(java.base@10.0.2/CompletableFuture.java:506)
   	at java.util.concurrent.CompletableFuture.complete(java.base@10.0.2/CompletableFuture.java:2073)
   	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.lambda$null$6(BookkeeperSchemaStorage.java:217)
   	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$$Lambda$1421/1611023719.apply(Unknown Source)
   	at java.util.concurrent.CompletableFuture.uniHandle(java.base@10.0.2/CompletableFuture.java:930)
   	at java.util.concurrent.CompletableFuture$UniHandle.tryFire(java.base@10.0.2/CompletableFuture.java:907)
   	at java.util.concurrent.CompletableFuture$Completion.exec(java.base@10.0.2/CompletableFuture.java:479)
   	at java.util.concurrent.ForkJoinTask.doExec(java.base@10.0.2/ForkJoinTask.java:290)
   	at java.util.concurrent.ForkJoinPool.runWorker(java.base@10.0.2/ForkJoinPool.java:1603)
   	at java.util.concurrent.ForkJoinWorkerThread.run(java.base@10.0.2/ForkJoinWorkerThread.java:177)
   
      Locked ownable synchronizers:
   	- <0x0000000624e2e9a0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
   ```
   
   After #13082, I think the issue has been fixed, but it only can fix the master branch and 2.10. For version < 2.10, I think we should change `topicPolicies.getMaxProducersPerTopic()` to only check the topic policies in the cache, it looks like `getIfPresent`. We should fix the branch-2.8 and branch-2.9.


-- 
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] hangc0276 commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928788414


   I doubt the write lock maybe not the root cause of this issue. 
   
   When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked. 
   
   So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`.


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-929951024


   [broker.jstack.txt](https://github.com/apache/pulsar/files/7250076/broker.jstack.txt)
   
   @codelipenghui FYI. The file has been desensitized.


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928677609


   > @wuzhanpeng Do you have the complete stack information? Not sure if there is dead lock in the metadata cache. Have you tried to restart the broker to see if the problem can be resolved.
   
   @codelipenghui Sometimes restarting the problem broker can solve it, and sometimes it will cause the same problem on other brokers(when bundles are transferred to other brokers, the pressure of handling producers will also be transferred). As I mentioned in the above description, restarting the broker frequently can easily trigger this problem.
   
   Because the complete jstack results of the production environment may contain some sensitive information, I am afraid that the full version cannot be uploaded. 😞 


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928677609


   > @wuzhanpeng Do you have the complete stack information? Not sure if there is dead lock in the metadata cache. Have you tried to restart the broker to see if the problem can be resolved.
   
   @codelipenghui Sometimes restarting the problem broker can solve it, and sometimes it will cause the same problem on other brokers(when bundles are transferred to other brokers, the pressure of handling producers will also be transferred). As I mentioned in the above description, restarting the broker frequently can easily trigger this problem.
   
   Because the complete jstack results of the production environment may contain some sensitive information, I am afraid that the full version cannot be uploaded. 😞 


-- 
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] hangc0276 commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-928788414


   I doubt the write lock maybe not the root cause of this issue. 
   
   When checking `isProducersExceeded`, it will load policy data from zk if topic policy not configured. Once the first thread call `isProducersExceeded`, the policy will be cached, and the following check won't be blocked. 
   
   So in my opinion, you'd better check the zk read latency or there are something wrong in `ZooKeeperDataCache`.


-- 
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] wuzhanpeng commented on pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#issuecomment-927439394


   @merlimat @sijie @eolivelli  Could you help to check this?


-- 
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] wuzhanpeng commented on a change in pull request #12202: Fix getting stuck when adding producers

Posted by GitBox <gi...@apache.org>.
wuzhanpeng commented on a change in pull request #12202:
URL: https://github.com/apache/pulsar/pull/12202#discussion_r717176613



##########
File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractTopic.java
##########
@@ -404,7 +406,9 @@ public String getReplicatorPrefix() {
                 .thenCompose(__ ->
                         incrementTopicEpochIfNeeded(producer, producerQueuedFuture))
                 .thenCompose(producerEpoch -> {
-                    lock.writeLock().lock();
+                    Lock producerLock = producer.getAccessMode() == ProducerAccessMode.Shared
+                            ? lock.readLock() : lock.writeLock();
+                    producerLock.lock();

Review comment:
       Thanks for your reply! Let me take the liberty to ask, what does `counter` refer to in `internalAddProducer`?




-- 
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