horizonzy opened a new pull request, #21159: URL: https://github.com/apache/pulsar/pull/21159
### Motivation The Auditor will start AuditorCheckAllLedgersTask to checkAllLedgers. It will iterate each ledger using `ledgerManager.asyncProcessLedgers`, then callback checkLedgersProcessor in the metadata store thread. https://github.com/apache/bookkeeper/blob/e8da8eb6cb7c8ef52336a51d4f7348ae07986c26/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/AuditorCheckAllLedgersTask.java#L158-L248 In the checkLedgersProcessor, it will invoke `ledgerUnderreplicationManager.isLedgerReplicationEnabled()`, it's a metadata store sync operation, so deadlock. There is the stack info: ``` "metadata-store-40-1" #161 prio=5 os_prio=31 cpu=43.17ms elapsed=24.19s tid=0x000000015932b000 nid=0x1720b waiting on condition [0x00000004d2be9000] java.lang.Thread.State: TIMED_WAITING (parking) at jdk.internal.misc.Unsafe.park([email protected]/Native Method) - parking to wait for <0x0000200002af57a8> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.parkNanos([email protected]/LockSupport.java:252) at java.util.concurrent.CompletableFuture$Signaller.block([email protected]/CompletableFuture.java:1866) at java.util.concurrent.ForkJoinPool.unmanagedBlock([email protected]/ForkJoinPool.java:3463) at java.util.concurrent.ForkJoinPool.managedBlock([email protected]/ForkJoinPool.java:3434) at java.util.concurrent.CompletableFuture.timedGet([email protected]/CompletableFuture.java:1939) at java.util.concurrent.CompletableFuture.get([email protected]/CompletableFuture.java:2095) at org.apache.pulsar.metadata.bookkeeper.PulsarLedgerUnderreplicationManager.isLedgerReplicationEnabled(PulsarLedgerUnderreplicationManager.java:735) at org.apache.bookkeeper.replication.AuditorCheckAllLedgersTask.lambda$checkAllLedgers$2(AuditorCheckAllLedgersTask.java:167) at org.apache.bookkeeper.replication.AuditorCheckAllLedgersTask$$Lambda$375/0x0000000800fd5340.process(Unknown Source) at org.apache.pulsar.metadata.bookkeeper.AbstractHierarchicalLedgerManager.lambda$asyncProcessLedgersInSingleNode$2(AbstractHierarchicalLedgerManager.java:211) at org.apache.pulsar.metadata.bookkeeper.AbstractHierarchicalLedgerManager$$Lambda$410/0x0000000800ffe9e8.accept(Unknown Source) at java.util.concurrent.CompletableFuture$UniAccept.tryFire([email protected]/CompletableFuture.java:718) at java.util.concurrent.CompletableFuture.postComplete([email protected]/CompletableFuture.java:510) at java.util.concurrent.CompletableFuture.complete([email protected]/CompletableFuture.java:2147) at org.apache.pulsar.metadata.impl.ZKMetadataStore.handleGetChildrenResult(ZKMetadataStore.java:291) at org.apache.pulsar.metadata.impl.ZKMetadataStore.lambda$batchOperation$5(ZKMetadataStore.java:222) at org.apache.pulsar.metadata.impl.ZKMetadataStore$$Lambda$296/0x0000000800f859f0.run(Unknown Source) at java.util.concurrent.Executors$RunnableAdapter.call([email protected]/Executors.java:539) at java.util.concurrent.FutureTask.run([email protected]/FutureTask.java:264) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run([email protected]/ScheduledThreadPoolExecutor.java:304) at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run([email protected]/Thread.java:833) ``` ### Modifications Refer to bookkeeper https://github.com/apache/bookkeeper/blob/e8da8eb6cb7c8ef52336a51d4f7348ae07986c26/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java#L607-L611. Using scheduler to trigger the processor. ### Documentation <!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. --> - [ ] `doc` <!-- Your PR contains doc changes. --> - [ ] `doc-required` <!-- Your PR changes impact docs and you will update later --> - [ ] `doc-not-needed` <!-- Your PR changes do not impact docs --> - [ ] `doc-complete` <!-- Docs have been already added --> ### Matching PR in forked repository PR in forked repository: <!-- ENTER URL HERE --> <!-- After opening this PR, the build in apache/pulsar will fail and instructions will be provided for opening a PR in the PR author's forked repository. apache/pulsar pull requests should be first tested in your own fork since the apache/pulsar CI based on GitHub Actions has constrained resources and quota. GitHub Actions provides separate quota for pull requests that are executed in a forked repository. The tests will be run in the forked repository until all PR review comments have been handled, the tests pass and the PR is approved by a reviewer. --> -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
