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

Fix the deadlock while using zookeeper thread to create ledger #13744

Merged
merged 3 commits into from Jan 14, 2022

Conversation

codelipenghui
Copy link
Contributor

@codelipenghui codelipenghui commented Jan 13, 2022

Motivation

Fixes: #13736, the deadlock when using ZK thread to create ledger.

The details of the deadlock:

Found one Java-level deadlock:
=============================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
  waiting to lock monitor 0x00007fa3fc033bd8 (object 0x00007fad5e804158, a org.apache.zookeeper.ZooKeeper$States),
  which is held by "PullMessageThread_167"
"PullMessageThread_167":
  waiting to lock monitor 0x00007fbe40026668 (object 0x00007fb596023c08, a java.util.concurrent.LinkedBlockingQueue),
  which is held by "ZKC-connect-executor-0-SendThread(9.142.172.233:2181)"

Java stack information for the threads listed above:
===================================================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1678)
	- waiting to lock <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:1905)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$10.zkRun(ZooKeeperClient.java:752)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.create(ZooKeeperClient.java:762)
	at org.apache.bookkeeper.util.ZkUtils.asyncCreateFullPathOptimistic(ZkUtils.java:75)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerIdImpl(ZkLedgerIdGenerator.java:78)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerId(ZkLedgerIdGenerator.java:73)
	at org.apache.bookkeeper.meta.LongZkLedgerIdGenerator.generateLedgerId(LongZkLedgerIdGenerator.java:301)
	at org.apache.bookkeeper.client.LedgerCreateOp.generateLedgerIdAndCreateLedger(LedgerCreateOp.java:194)
	at org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:182)
	at org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:860)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncCreateLedger(ManagedLedgerImpl.java:3645)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createLedgerAfterClosed(ManagedLedgerImpl.java:1596)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1587)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:236)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:552)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1557/731657311.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$3(LedgerHandle.java:614)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1563/295431727.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.MetadataUpdateLoop.lambda$writeLoop$1(MetadataUpdateLoop.java:161)
	at org.apache.bookkeeper.client.MetadataUpdateLoop$$Lambda$1562/765900306.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager$4.processResult(AbstractZkLedgerManager.java:508)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$22$1.processResult(ZooKeeperClient.java:1094)
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:638)
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:541)
	- locked <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.access$2600(ClientCnxn.java:106)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1403)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1331)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1309)
"PullMessageThread_167":
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:537)
	- waiting to lock <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1680)
	- locked <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2411)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19.zkRun(ZooKeeperClient.java:1009)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.getData(ZooKeeperClient.java:1019)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:435)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:430)
	at org.apache.bookkeeper.meta.CleanupLedgerManager.readLedgerMetadata(CleanupLedgerManager.java:157)
	at org.apache.bookkeeper.client.LedgerOpenOp.initiate(LedgerOpenOp.java:114)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.open(LedgerOpenOp.java:269)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.execute(LedgerOpenOp.java:247)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$getLedgerHandle$20(ManagedLedgerImpl.java:1773)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$$Lambda$1195/876838597.apply(Unknown Source)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap$Section.put(ConcurrentLongHashMap.java:287)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap.computeIfAbsent(ConcurrentLongHashMap.java:135)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.getLedgerHandle(ManagedLedgerImpl.java:1742)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntry(ManagedLedgerImpl.java:1839)
	at org.apache.bookkeeper.mledger.impl.OpFindNewest.find(OpFindNewest.java:147)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncFindPosition(ManagedLedgerImpl.java:1670)
	at org.streamnative.pulsar.handlers.rocketmq.utils.MessageIdUtils.getPositionForOffset(MessageIdUtils.java:183)
	at org.streamnative.pulsar.handlers.rocketmq.inner.RopServerCnx.getMessage(RopServerCnx.java:692)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:359)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:170)
	at org.streamnative.pulsar.handlers.rocketmq.inner.proxy.RopBrokerProxy$PullMessageProcessorProxy.processRequest(RopBrokerProxy.java:1098)
	at org.streamnative.pulsar.handlers.rocketmq.inner.NettyRemotingAbstract$1.run(NettyRemotingAbstract.java:202)
	at org.apache.rocketmq.remoting.netty.RequestTask.run(RequestTask.java:80)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	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)

Found 1 deadlock.

Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.

Documentation

Check the box below or label this PR directly (if you have committer privilege).

Need to update docs?

  • doc-required

    (If you need help on updating docs, create a doc issue)

  • no-need-doc

    (Please explain why)

  • doc

    (If this PR contains doc changes)

Copy link
Member

@wolfstudy wolfstudy left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cool, LGTM +1

### Motivation

Fixes: apache#13736, the deadlock when using ZK thread to create ledger.

The details of the deadlock:

```
Found one Java-level deadlock:
=============================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
  waiting to lock monitor 0x00007fa3fc033bd8 (object 0x00007fad5e804158, a org.apache.zookeeper.ZooKeeper$States),
  which is held by "PullMessageThread_167"
"PullMessageThread_167":
  waiting to lock monitor 0x00007fbe40026668 (object 0x00007fb596023c08, a java.util.concurrent.LinkedBlockingQueue),
  which is held by "ZKC-connect-executor-0-SendThread(9.142.172.233:2181)"

Java stack information for the threads listed above:
===================================================
"ZKC-connect-executor-0-SendThread(9.142.172.233:2181)":
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1678)
	- waiting to lock <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:1905)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$10.zkRun(ZooKeeperClient.java:752)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.create(ZooKeeperClient.java:762)
	at org.apache.bookkeeper.util.ZkUtils.asyncCreateFullPathOptimistic(ZkUtils.java:75)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerIdImpl(ZkLedgerIdGenerator.java:78)
	at org.apache.bookkeeper.meta.ZkLedgerIdGenerator.generateLedgerId(ZkLedgerIdGenerator.java:73)
	at org.apache.bookkeeper.meta.LongZkLedgerIdGenerator.generateLedgerId(LongZkLedgerIdGenerator.java:301)
	at org.apache.bookkeeper.client.LedgerCreateOp.generateLedgerIdAndCreateLedger(LedgerCreateOp.java:194)
	at org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:182)
	at org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:860)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncCreateLedger(ManagedLedgerImpl.java:3645)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createLedgerAfterClosed(ManagedLedgerImpl.java:1596)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1587)
	- locked <0x00007fadbdfa7b90> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl)
	at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:236)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:552)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1557/731657311.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$3(LedgerHandle.java:614)
	at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$1563/295431727.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.client.MetadataUpdateLoop.lambda$writeLoop$1(MetadataUpdateLoop.java:161)
	at org.apache.bookkeeper.client.MetadataUpdateLoop$$Lambda$1562/765900306.accept(Unknown Source)
	at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
	at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager$4.processResult(AbstractZkLedgerManager.java:508)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$22$1.processResult(ZooKeeperClient.java:1094)
	at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:638)
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:541)
	- locked <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.access$2600(ClientCnxn.java:106)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanup(ClientCnxn.java:1403)
	at org.apache.zookeeper.ClientCnxn$SendThread.cleanAndNotifyState(ClientCnxn.java:1331)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1309)
"PullMessageThread_167":
	at org.apache.zookeeper.ClientCnxn$EventThread.queuePacket(ClientCnxn.java:537)
	- waiting to lock <0x00007fb596023c08> (a java.util.concurrent.LinkedBlockingQueue)
	at org.apache.zookeeper.ClientCnxn.finishPacket(ClientCnxn.java:781)
	at org.apache.zookeeper.ClientCnxn.conLossPacket(ClientCnxn.java:818)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1680)
	- locked <0x00007fad5e804158> (a org.apache.zookeeper.ZooKeeper$States)
	at org.apache.zookeeper.ClientCnxn.queuePacket(ClientCnxn.java:1649)
	at org.apache.zookeeper.ZooKeeper.getData(ZooKeeper.java:2411)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$19.zkRun(ZooKeeperClient.java:1009)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient$ZkRetryRunnable.run(ZooKeeperClient.java:392)
	at org.apache.bookkeeper.zookeeper.ZooKeeperClient.getData(ZooKeeperClient.java:1019)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:435)
	at org.apache.bookkeeper.meta.AbstractZkLedgerManager.readLedgerMetadata(AbstractZkLedgerManager.java:430)
	at org.apache.bookkeeper.meta.CleanupLedgerManager.readLedgerMetadata(CleanupLedgerManager.java:157)
	at org.apache.bookkeeper.client.LedgerOpenOp.initiate(LedgerOpenOp.java:114)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.open(LedgerOpenOp.java:269)
	at org.apache.bookkeeper.client.LedgerOpenOp$OpenBuilderImpl.execute(LedgerOpenOp.java:247)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.lambda$getLedgerHandle$20(ManagedLedgerImpl.java:1773)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$$Lambda$1195/876838597.apply(Unknown Source)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap$Section.put(ConcurrentLongHashMap.java:287)
	at org.apache.pulsar.common.util.collections.ConcurrentLongHashMap.computeIfAbsent(ConcurrentLongHashMap.java:135)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.getLedgerHandle(ManagedLedgerImpl.java:1742)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncReadEntry(ManagedLedgerImpl.java:1839)
	at org.apache.bookkeeper.mledger.impl.OpFindNewest.find(OpFindNewest.java:147)
	at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncFindPosition(ManagedLedgerImpl.java:1670)
	at org.streamnative.pulsar.handlers.rocketmq.utils.MessageIdUtils.getPositionForOffset(MessageIdUtils.java:183)
	at org.streamnative.pulsar.handlers.rocketmq.inner.RopServerCnx.getMessage(RopServerCnx.java:692)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:359)
	at org.streamnative.pulsar.handlers.rocketmq.inner.processor.PullMessageProcessor.processRequest(PullMessageProcessor.java:170)
	at org.streamnative.pulsar.handlers.rocketmq.inner.proxy.RopBrokerProxy$PullMessageProcessorProxy.processRequest(RopBrokerProxy.java:1098)
	at org.streamnative.pulsar.handlers.rocketmq.inner.NettyRemotingAbstract$1.run(NettyRemotingAbstract.java:202)
	at org.apache.rocketmq.remoting.netty.RequestTask.run(RequestTask.java:80)
	at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	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)

Found 1 deadlock.
```

### Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.
@codelipenghui codelipenghui merged commit 1d4c374 into apache:master Jan 14, 2022
@codelipenghui codelipenghui deleted the penghui/fix-13736 branch January 14, 2022 09:47
codelipenghui added a commit that referenced this pull request Jan 18, 2022
### Motivation

Fixes: #13736, the deadlock when using ZK thread to create a ledger.

### Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.

(cherry picked from commit 1d4c374)
@codelipenghui codelipenghui added the cherry-picked/branch-2.9 Archived: 2.9 is end of life label Jan 18, 2022
codelipenghui added a commit that referenced this pull request Jan 18, 2022
### Motivation

Fixes: #13736, the deadlock when using ZK thread to create a ledger.

### Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.

(cherry picked from commit 1d4c374)
@codelipenghui codelipenghui added the cherry-picked/branch-2.8 Archived: 2.8 is end of life label Jan 18, 2022
lhotari pushed a commit to datastax/pulsar that referenced this pull request Jan 20, 2022
…e#13744)

### Motivation

Fixes: apache#13736, the deadlock when using ZK thread to create a ledger.

### Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.

(cherry picked from commit 1d4c374)
(cherry picked from commit d7117fb)
@gaoran10 gaoran10 added area/broker type/bug The PR fixed a bug or issue reported a bug labels Jan 25, 2022
@codelipenghui codelipenghui restored the penghui/fix-13736 branch May 17, 2022 01:20
@codelipenghui codelipenghui deleted the penghui/fix-13736 branch May 17, 2022 01:28
Shawyeok pushed a commit to Shawyeok/pulsar that referenced this pull request Sep 6, 2022
…e#13744)

### Motivation

Fixes: apache#13736, the deadlock when using ZK thread to create a ledger.

### Modification

Use the executor of the managed ledger to create the ledger to avoid the deadlock.

(cherry picked from commit 1d4c374)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/broker cherry-picked/branch-2.8 Archived: 2.8 is end of life cherry-picked/branch-2.9 Archived: 2.9 is end of life doc-not-needed Your PR changes do not impact docs release/2.8.3 release/2.9.2 type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[DeadLock] Attempting to create a new ledger when the ledger is closed may result in a deadlock
8 participants