massakam commented on issue #4635: Bookie down causes deadlock in broker
URL: https://github.com/apache/pulsar/issues/4635#issuecomment-508150928
 
 
   However, `pulsar-ordered-OrderedExecutor-2-0-EventThread` seems to block 
itself until a timeout occurs.
   ```
   "pulsar-ordered-OrderedExecutor-2-0-EventThread" daemon prio=5 tid=23 
TIMED_WAITING
   java.lang.Thread.State: TIMED_WAITING
           at sun.misc.Unsafe.park(Native Method)
           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.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:95)
           at 
org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.getRack(ZkBookieRackAffinityMapping.java:154)
           at 
org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.resolve(ZkBookieRackAffinityMapping.java:146)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl$DNSResolverDecorator.resolve(RackawareEnsemblePlacementPolicyImpl.java:174)
           at 
org.apache.bookkeeper.net.NetUtils.resolveNetworkLocation(NetUtils.java:81)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.resolveNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:402)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.createBookieNode(RackawareEnsemblePlacementPolicyImpl.java:259)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.convertBookiesToNodes(RackawareEnsemblePlacementPolicyImpl.java:512)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal(RackawareEnsemblePlacementPolicyImpl.java:575)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal(RackawareEnsemblePlacementPolicyImpl.java:540)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsemble(RackawareEnsemblePlacementPolicyImpl.java:531)
           at 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.newEnsemble(RackawareEnsemblePlacementPolicy.java:98)
           at 
org.apache.bookkeeper.client.BookieWatcherImpl.newEnsemble(BookieWatcherImpl.java:233)
           at 
org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:141)
           at 
org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:831)
           at 
org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncCreateLedger(ManagedLedgerImpl.java:3046)
           at 
org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1388)
           at 
org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:192)
           at 
org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:554)
           at 
org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$828/960239181.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.complete(CompletableFuture.java:1962)
           at 
org.apache.bookkeeper.common.concurrent.FutureUtils.complete(FutureUtils.java:126)
           at 
org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$3(LedgerHandle.java:614)
           at 
org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$833/911729943.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.complete(CompletableFuture.java:1962)
           at 
org.apache.bookkeeper.client.MetadataUpdateLoop.lambda$writeLoop$1(MetadataUpdateLoop.java:129)
           at 
org.apache.bookkeeper.client.MetadataUpdateLoop$$Lambda$825/2131232665.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.complete(CompletableFuture.java:1962)
           at 
org.apache.bookkeeper.meta.AbstractZkLedgerManager$4.processResult(AbstractZkLedgerManager.java:450)
           at 
org.apache.bookkeeper.zookeeper.ZooKeeperClient$22$1.processResult(ZooKeeperClient.java:1091)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:557)
           at 
org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
   ```
   
   In `AbstractZkLedgerManager`, the ZK event callback thread does the 
subsequent task without starting a different thread.
   
https://github.com/apache/bookkeeper/blob/release-4.9.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/meta/AbstractZkLedgerManager.java#L441-L450
   
   Is this similar to the part that caused the previous bug?
   
https://github.com/apache/pulsar/pull/3591/files#diff-e31620999c95d63a09c11ae894f943be

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to