Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

sometimes internalDeleteTopicForcefully will block forever #14438

Closed
leizhiyuan opened this issue Feb 24, 2022 · 12 comments
Closed

sometimes internalDeleteTopicForcefully will block forever #14438

leizhiyuan opened this issue Feb 24, 2022 · 12 comments
Assignees
Labels
release/2.7.5 release/2.8.4 release/2.9.3 type/bug The PR fixed a bug or issue reported a bug
Milestone

Comments

@leizhiyuan
Copy link
Contributor

pulsar().getBrokerService().deleteTopic(topicName.toString(), true, deleteSchema).get();

"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

@michaeljmarshall
Copy link
Member

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

@michaeljmarshall
Copy link
Member

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

execute(() -> {
Code code = Code.get(rc);
if (code == Code.OK) {
future.complete(true);
} else if (code == Code.NONODE) {
future.complete(false);
} else {
future.completeExceptionally(getException(code, path));
}
}, future);
.

In 2.8, we added that execute method call, which schedules exists callbacks to run on this single threaded executor:

this.executor = Executors
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("metadata-store"));

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)

@michaeljmarshall
Copy link
Member

michaeljmarshall commented Feb 25, 2022

Based on that thread dump, it looks like the blocking is coming from this code:

getBrokerService().fetchPartitionedTopicMetadataAsync(TopicName.get(topicName.getPartitionedTopicName()))
.thenAccept((metadata -> {
// make sure all sub partitions were deleted
for (int i = 0; i < metadata.partitions; i++) {
if (brokerService.getPulsar().getPulsarResources().getTopicResources()
.persistentTopicExists(topicName.getPartition(i)).join()) {

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

@eolivelli
Copy link
Contributor

good work @michaeljmarshall

We should not call 'join' and chain the async functions calls to persistentTopicExists.

I agree that we should fix this.

@eolivelli
Copy link
Contributor

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

@mattisonchao
Copy link
Member

mattisonchao commented Feb 25, 2022

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

@michaeljmarshall
Copy link
Member

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

@michaeljmarshall
Copy link
Member

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

@leizhiyuan
Copy link
Contributor Author

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

@michaeljmarshall
Copy link
Member

@leizhiyuan - thanks for sharing that thread dump. I am intrigued that the metadata-store thread that I referenced #14438 (comment) 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.

@michaeljmarshall
Copy link
Member

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

@leizhiyuan
Copy link
Contributor Author

@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

image

when cache missed, we do this.
image

codelipenghui pushed a commit that referenced this issue Mar 1, 2022
Master issue #14438

### Motivation

Invoking the ``join()`` method in the async method will cause some deadlock. 

### Modifications

- Refactor ``PersistentTopic#tryToDeletePartitionedMetadata`` to pure async.
codelipenghui pushed a commit that referenced this issue Mar 1, 2022
Master issue #14438

### Motivation

Invoking the ``join()`` method in the async method will cause some deadlock.

### Modifications

- Refactor ``PersistentTopic#tryToDeletePartitionedMetadata`` to pure async.

(cherry picked from commit 65318e8)
gaoran10 pushed a commit that referenced this issue Mar 1, 2022
Master issue #14438

### Motivation

Invoking the ``join()`` method in the async method will cause some deadlock.

### Modifications

- Refactor ``PersistentTopic#tryToDeletePartitionedMetadata`` to pure async.

(cherry picked from commit 65318e8)
@codelipenghui codelipenghui added this to the 2.10.0 milestone Mar 15, 2022
Nicklee007 pushed a commit to Nicklee007/pulsar that referenced this issue Apr 20, 2022
Master issue apache#14438

### Motivation

Invoking the ``join()`` method in the async method will cause some deadlock. 

### Modifications

- Refactor ``PersistentTopic#tryToDeletePartitionedMetadata`` to pure async.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
release/2.7.5 release/2.8.4 release/2.9.3 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

7 participants