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 2022/02/24 05:23:59 UTC

[GitHub] [pulsar] leizhiyuan opened a new issue #14438: sometimes internalDeleteTopicForcefully will block forever

leizhiyuan opened a new issue #14438:
URL: https://github.com/apache/pulsar/issues/14438


   https://github.com/apache/pulsar/blob/d1fb88acace2ceefc234458d2c0d93f2844c7a54/pulsar-broker/src/main/java/org/apache/pulsar/broker/admin/impl/PersistentTopicsBase.java#L312
   
   ```
   "pulsar-web-44-4" #148 prio=5 os_prio=0 tid=0x00007fcbf7f89000 nid=0x6d97 waiting on condition [0x00007fcabc99e000]
      java.lang.Thread.State: WAITING (parking)
   	at sun.misc.Unsafe.park(Native Method)
   	- parking to wait for  <0x000000073d14e618> (a java.util.concurrent.CompletableFuture$Signaller)
   	at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
   	at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1693)
   	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
   	at java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1729)
   	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
   	at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.internalDeleteTopicForcefully(PersistentTopicsBase.java:379)
   	at org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.internalDeleteTopic(PersistentTopicsBase.java:961)
   	at org.apache.pulsar.broker.admin.v2.PersistentTopics.deleteTopic(PersistentTopics.java:889)
   	at sun.reflect.GeneratedMethodAccessor241.invoke(Unknown Source)
   	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.lang.reflect.Method.invoke(Method.java:498)
   	at org.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory.lambda$static$0(ResourceMethodInvocationHandlerFactory.java:52)
   	at org.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory$$Lambda$419/1877964230.invoke(Unknown Source)
   	at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher$1.run(AbstractJavaResourceMethodDispatcher.java:124)
   	at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.invoke(AbstractJavaResourceMethodDispatcher.java:167)
   	at org.glassfish.jersey.server.model.internal.JavaResourceMethodDispatcherProvider$VoidOutInvoker.doDispatch(JavaResourceMethodDispatcherProvider.java:159)
   	at org.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.dispatch(AbstractJavaResourceMethodDispatcher.java:79)
   	at org.glassfish.jersey.server.model.ResourceMethodInvoker.invoke(ResourceMethodInvoker.java:475)
   	at org.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:397)
   	at org.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:81)
   	at org.glassfish.jersey.server.ServerRuntime$1.run(ServerRuntime.java:255)
   	at org.glassfish.jersey.internal.Errors$1.call(Errors.java:248)
   	at org.glassfish.jersey.internal.Errors$1.call(Errors.java:244)
   	at org.glassfish.jersey.internal.Errors.process(Errors.java:292)
   	at org.glassfish.jersey.internal.Errors.process(Errors.java:274)
   	at org.glassfish.jersey.internal.Errors.process(Errors.java:244)
   	at org.glassfish.jersey.process.internal.RequestScope.runInScope(RequestScope.java:265)
   	at org.glassfish.jersey.server.ServerRuntime.process(ServerRuntime.java:234)
   	at org.glassfish.jersey.server.ApplicationHandler.handle(ApplicationHandler.java:680)
   	at org.glassfish.jersey.servlet.WebComponent.serviceImpl(WebComponent.java:394)
   	at org.glassfish.jersey.servlet.WebComponent.service(WebComponent.java:346)
   	at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:366)
   	at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:319)
   	at org.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:205)
   	at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:799)
   	at org.eclipse.jetty.servlet.ServletHandler$ChainEnd.doFilter(ServletHandler.java:1631)
   	at org.apache.pulsar.broker.web.VpcLookupFilter.doFilter(VpcLookupFilter.java:148)
   	at org.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
   	at org.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1601)
   	at org.apache.pulsar.broker.web.ResponseHandlerFilter.doFilter(ResponseHandlerFilter.java:66)
   	at org.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
   	at org.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1601)
   	at org.apache.pulsar.broker.web.AuthenticationFilter.doFilter(AuthenticationFilter.java:82)
   	at org.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
   	at org.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1601)
   	at org.apache.pulsar.broker.web.PreInterceptFilter.doFilter(PreInterceptFilter.java:68)
   	at org.eclipse.jetty.servlet.FilterHolder.doFilter(FilterHolder.java:193)
   	at org.eclipse.jetty.servlet.ServletHandler$Chain.doFilter(ServletHandler.java:1601)
   	at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:548)
   	at org.eclipse.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:233)
   	at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:1624)
   	at org.eclipse.jetty.server.handler.ScopedHandler.nextHandle(ScopedHandler.java:233)
   	at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1434)
   	at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:188)
   	at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:501)
   	at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:1594)
   	at org.eclipse.jetty.server.handler.ScopedHandler.nextScope(ScopedHandler.java:186)
   	at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1349)
   	at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
   	at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:234)
   	at org.eclipse.jetty.server.handler.HandlerCollection.handle(HandlerCollection.java:146)
   	at org.eclipse.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:179)
   	at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:127)
   	at org.eclipse.jetty.server.Server.handle(Server.java:516)
   	at org.eclipse.jetty.server.HttpChannel.lambda$handle$1(HttpChannel.java:400)
   	at org.eclipse.jetty.server.HttpChannel$$Lambda$537/2098987772.dispatch(Unknown Source)
   	at org.eclipse.jetty.server.HttpChannel.dispatch(HttpChannel.java:645)
   	at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:392)
   	at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:277)
   	at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:311)
   	at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:105)
   	at org.eclipse.jetty.io.ChannelEndPoint$1.run(ChannelEndPoint.java:104)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
   	at java.lang.Thread.run(Thread.java:748)
   ```
   
   in our scene, when we dump the thread many times , we can see this will wait forever


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1051371387


   @leizhiyuan - note that the bug I'm seeing is triggered by `brokerDeleteInactivePartitionedTopicMetadataEnabled=true`. That is the only way that `tryToDeletePartitionedMetadata` can result in a deadlock scenario. If you're running with that configuration set to `false`, then maybe there are multiple cases of deadlock here.


-- 
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] leizhiyuan commented on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
leizhiyuan commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1051393501


   > @leizhiyuan - it'd be valuable to know what your `metadata-store` thread stack says.
   [27151.log](https://github.com/apache/pulsar/files/8145521/27151.log)
   
   


-- 
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] eolivelli commented on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050601917


   good work @michaeljmarshall 
   
   We should not call 'join' and chain the async functions calls to persistentTopicExists.
   
   I agree that we should fix 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] eolivelli commented on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
eolivelli commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050616874


   @leizhiyuan I will send a PR for this if you don't have time


-- 
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 closed issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
codelipenghui closed issue #14438:
URL: https://github.com/apache/pulsar/issues/14438


   


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1054722361


   @leizhiyuan - after looking through your thread dump a bit more, I am wondering if you have a separate issue in your delayed message implementation? I see that thread is blocked on some other thread that is completing a lookup call (that lookup call has a timeout on it, so that thread isn't indefinitely blocked). Is there any chance that the `"DM-sync-delay-index"` thread being blocked is causing an issue here?
   
   ```
   "DM-sync-delay-index" #223 daemon prio=5 os_prio=0 tid=0x00007fcbd8023800 nid=0x7cb5 waiting for monitor entry [0x00007fcaa193f000]
      java.lang.Thread.State: BLOCKED (on object monitor)
           at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1868)
           - waiting to lock <0x0000000760a001d8> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2037)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2020)
           at com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:112)
           at com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:67)
           at com.tencent.tdmq.delaymessage.inner.MetadataCache.isDelayTenantOwner(MetadataCache.java:151)
           at com.tencent.tdmq.delaymessage.inner.SyncIndexService.syncTenant(SyncIndexService.java:65)
           at com.tencent.tdmq.delaymessage.inner.SyncIndexService.syncAllTenants(SyncIndexService.java:52)
           at com.tencent.tdmq.delaymessage.inner.SyncIndexService.lambda$new$1(SyncIndexService.java:41)
           at com.tencent.tdmq.delaymessage.inner.SyncIndexService$$Lambda$832/1549413052.run(Unknown Source)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
           at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
           at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   ```
   
   ```
   "DM-clear-thread" #227 daemon prio=5 os_prio=0 tid=0x00007fcbd8035800 nid=0x7cb9 waiting on condition [0x00007fcaa153a000]
      java.lang.Thread.State: TIMED_WAITING (parking)
           at sun.misc.Unsafe.park(Native Method)
           - parking to wait for  <0x0000000760a08240> (a java.util.concurrent.CompletableFuture$Signaller)
           at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
           at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1695)
           at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
           at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1775)
           at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
           at org.apache.pulsar.client.admin.internal.LookupImpl.lookupTopic(LookupImpl.java:56)
           at com.tencent.tdmq.delaymessage.inner.MetadataCache.lookup(MetadataCache.java:166)
           at com.tencent.tdmq.delaymessage.inner.MetadataCache$$Lambda$840/1429314972.load(Unknown Source)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:3366)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache$$Lambda$839/2030879142.apply(Unknown Source)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2039)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$88/947462790.apply(Unknown Source)
           at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853)
           - locked <0x0000000760a001d8> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2037)
           at com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2020)
           at com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:112)
           at com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:67)
           at com.tencent.tdmq.delaymessage.inner.MetadataCache.isDelayTenantOwner(MetadataCache.java:151)
           at com.tencent.tdmq.delaymessage.inner.ClearTask.check(ClearTask.java:67)
           at com.tencent.tdmq.delaymessage.inner.ClearTask$$Lambda$846/1309049241.run(Unknown Source)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
           at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
           at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   ```


-- 
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] leizhiyuan commented on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
leizhiyuan commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1054910918


   > @leizhiyuan - after looking through your thread dump a bit more, I am wondering if you have a separate issue in your delayed message implementation? I see that thread is blocked on some other thread that is completing a lookup call (that lookup call has a timeout on it, so that thread isn't indefinitely blocked). Is there any chance that the `"DM-sync-delay-index"` thread being blocked is causing an issue here?
   > 
   > ```
   > "DM-sync-delay-index" #223 daemon prio=5 os_prio=0 tid=0x00007fcbd8023800 nid=0x7cb5 waiting for monitor entry [0x00007fcaa193f000]
   >    java.lang.Thread.State: BLOCKED (on object monitor)
   >         at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1868)
   >         - waiting to lock <0x0000000760a001d8> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2037)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2020)
   >         at com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:112)
   >         at com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:67)
   >         at com.tencent.tdmq.delaymessage.inner.MetadataCache.isDelayTenantOwner(MetadataCache.java:151)
   >         at com.tencent.tdmq.delaymessage.inner.SyncIndexService.syncTenant(SyncIndexService.java:65)
   >         at com.tencent.tdmq.delaymessage.inner.SyncIndexService.syncAllTenants(SyncIndexService.java:52)
   >         at com.tencent.tdmq.delaymessage.inner.SyncIndexService.lambda$new$1(SyncIndexService.java:41)
   >         at com.tencent.tdmq.delaymessage.inner.SyncIndexService$$Lambda$832/1549413052.run(Unknown Source)
   >         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   >         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
   >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
   >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
   >         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   >         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   >         at java.lang.Thread.run(Thread.java:748)
   > ```
   > 
   > ```
   > "DM-clear-thread" #227 daemon prio=5 os_prio=0 tid=0x00007fcbd8035800 nid=0x7cb9 waiting on condition [0x00007fcaa153a000]
   >    java.lang.Thread.State: TIMED_WAITING (parking)
   >         at sun.misc.Unsafe.park(Native Method)
   >         - parking to wait for  <0x0000000760a08240> (a java.util.concurrent.CompletableFuture$Signaller)
   >         at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
   >         at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1695)
   >         at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
   >         at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1775)
   >         at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1915)
   >         at org.apache.pulsar.client.admin.internal.LookupImpl.lookupTopic(LookupImpl.java:56)
   >         at com.tencent.tdmq.delaymessage.inner.MetadataCache.lookup(MetadataCache.java:166)
   >         at com.tencent.tdmq.delaymessage.inner.MetadataCache$$Lambda$840/1429314972.load(Unknown Source)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:3366)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache$$Lambda$839/2030879142.apply(Unknown Source)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$14(BoundedLocalCache.java:2039)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache$$Lambda$88/947462790.apply(Unknown Source)
   >         at java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853)
   >         - locked <0x0000000760a001d8> (a java.util.concurrent.ConcurrentHashMap$ReservationNode)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:2037)
   >         at com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:2020)
   >         at com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:112)
   >         at com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:67)
   >         at com.tencent.tdmq.delaymessage.inner.MetadataCache.isDelayTenantOwner(MetadataCache.java:151)
   >         at com.tencent.tdmq.delaymessage.inner.ClearTask.check(ClearTask.java:67)
   >         at com.tencent.tdmq.delaymessage.inner.ClearTask$$Lambda$846/1309049241.run(Unknown Source)
   >         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
   >         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
   >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
   >         at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
   >         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   >         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   >         at java.lang.Thread.run(Thread.java:748)
   > ```
   
   Thanks for your help, but we analyzed this thread, `DM-sync-delay-index` will not affect the lookup process. It only uses cache to lookup if a topic exists, if cache missed ,It will lookup use `PulsarService`
   
   
   <img width="922" alt="image" src="https://user-images.githubusercontent.com/2684384/156091563-b10c79f2-e1ed-44de-a247-0f104147c9ca.png">
   
   when cache missed, we do this.
   <img width="1006" alt="image" src="https://user-images.githubusercontent.com/2684384/156091601-311364f2-4dbc-4ae7-9dce-47806c39711b.png">
   
   


-- 
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] mattisonchao commented on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
mattisonchao commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050628564


   @eolivelli  Sorry I missed your message, I have already pushed a PR. #14469 


-- 
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 edited a comment on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall edited a comment on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050576074


   Based on that thread dump, it looks like the blocking is coming from this code:
   
   https://github.com/apache/pulsar/blob/0ef7baa5131f0dee4a57e61eae1d7686f7f60f1e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L2279-L2284
   
   When the zk thread completes the future returned by `fetchPartitionedTopicMetadataAsync`, it completes the callback on the `metadata-store` thread. That callback is then going to lead to a call to metadata store, which needs to run the callback on the `metadata-store` thread, but that thread is already in use.
   
   The fundamental issue here is deadlock.
   
   @merlimat @lhotari @codelipenghui @eolivelli - I think we should seriously consider solving this deadlock scenario for 2.10.0.
   
   EDIT: added reference to 2.10.0


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1051372251


   @leizhiyuan - it'd be valuable to know what your `metadata-store` thread stack says.


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050574560


   @leizhiyuan - in your thread dump, what is the stack for the thread named `metadata-store-x-x` where `x` is a number? If I had to guess, that thread is in a `WAITING` state, and is the reason that the future will never return. In master, here are the lines that are causing the issue: https://github.com/apache/pulsar/blob/0ef7baa5131f0dee4a57e61eae1d7686f7f60f1e/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/ZKMetadataStore.java#L310-L319.
   
   In 2.8, we added that `execute` method call, which schedules `exists` callbacks to run on this single threaded executor: https://github.com/apache/pulsar/blob/0ef7baa5131f0dee4a57e61eae1d7686f7f60f1e/pulsar-metadata/src/main/java/org/apache/pulsar/metadata/impl/AbstractMetadataStore.java#L78-L79
   
   If that executor is not handling requests, it will not complete futures, and which will lead to the problem you're seeing here.
   
   In my case, I can see the following stack for my metadata-store thread:
   
   ```
   "metadata-store-6-1" - Thread t@22
      java.lang.Thread.State: WAITING
           at java.base@11.0.11/jdk.internal.misc.Unsafe.park(Native Method)
           - parking to wait for <16ddd841> (a java.util.concurrent.CompletableFuture$Signaller)
           at java.base@11.0.11/java.util.concurrent.locks.LockSupport.park(LockSupport.java:194)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1796)
           at java.base@11.0.11/java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3128)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture.waitingGet(CompletableFuture.java:1823)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture.join(CompletableFuture.java:2043)
           at app//org.apache.pulsar.broker.service.persistent.PersistentTopic.lambda$tryToDeletePartitionedMetadata$96(PersistentTopic.java:2267)
           at app//org.apache.pulsar.broker.service.persistent.PersistentTopic$$Lambda$1354/0x0000000840851c40.accept(Unknown Source)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506)
           at java.base@11.0.11/java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073)
           at app//org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$null$7(ZKMetadataStore.java:139)
           at app//org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$234/0x0000000840343840.run(Unknown Source)
           at java.base@11.0.11/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
           at java.base@11.0.11/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
           at app//io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
           at java.base@11.0.11/java.lang.Thread.run(Thread.java:829)
   
      Locked ownable synchronizers:
           - locked <37622120> (a java.util.concurrent.ThreadPoolExecutor$Worker)
   ```


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1051709718


   @leizhiyuan - thanks for sharing that thread dump. I am intrigued that the `metadata-store` thread that I referenced https://github.com/apache/pulsar/issues/14438#issuecomment-1050574560 is not in your thread dump. Which version of Pulsar are you running and what is your configured metadata store?
   
   I might have been wrong in my initial analysis in thinking we had the same issues.


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050561056


   @leizhiyuan - which version of pulsar are you seeing this error? Have you reproduced the problem against all of the versions taht @315157973 lists here?


-- 
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 issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
michaeljmarshall commented on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050576074


   Based on that thread dump, it looks like the blocking is coming from this code:
   
   https://github.com/apache/pulsar/blob/0ef7baa5131f0dee4a57e61eae1d7686f7f60f1e/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java#L2279-L2284
   
   When the zk thread completes the future returned by `fetchPartitionedTopicMetadataAsync`, it completes the callback on the `metadata-store` thread. That callback is then going to lead to a call to metadata store, which needs to run the callback on the `metadata-store` thread, but that thread is already in use.
   
   The fundamental issue here is deadlock.
   
   @merlimat @lhotari @codelipenghui @eolivelli - I think we should seriously consider solving this deadlock scenario.


-- 
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] mattisonchao edited a comment on issue #14438: sometimes internalDeleteTopicForcefully will block forever

Posted by GitBox <gi...@apache.org>.
mattisonchao edited a comment on issue #14438:
URL: https://github.com/apache/pulsar/issues/14438#issuecomment-1050628564


   Sorry @eolivelli, I missed your message, I have already pushed a PR. #14469 


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