pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] massakam commented on issue #3566: Unexpected ZooKeeper disconnection and finally Brokers shutdown
Date Wed, 13 Feb 2019 13:12:10 GMT
massakam commented on issue #3566: Unexpected ZooKeeper disconnection and finally Brokers shutdown
URL: https://github.com/apache/pulsar/issues/3566#issuecomment-463193894
 
 
   I think the cause of this problem is that thread `pulsar-ordered-OrderedExecutor-x-x-EventThread`
is deadlocking itself.
   
   ```
   "pulsar-ordered-OrderedExecutor-7-0-EventThread" daemon prio=5 tid=26 in Object.wait()
   java.lang.Thread.State: WAITING (on object monitor)
           at sun.misc.Unsafe.park(Native Method)
           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.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:94)
           at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.getRack(ZkBookieRackAffinityMapping.java:153)
           at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.resolve(ZkBookieRackAffinityMapping.java:145)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl$DNSResolverDecorator.resolve(RackawareEnsemblePlacementPolicyImpl.java:153)
           at org.apache.bookkeeper.net.NetUtils.resolveNetworkLocation(NetUtils.java:81)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.resolveNetworkLocation(RackawareEnsemblePlacementPolicyImpl.java:342)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.createBookieNode(RackawareEnsemblePlacementPolicyImpl.java:221)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.convertBookiesToNodes(RackawareEnsemblePlacementPolicyImpl.java:452)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal(RackawareEnsemblePlacementPolicyImpl.java:515)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal(RackawareEnsemblePlacementPolicyImpl.java:480)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsemble(RackawareEnsemblePlacementPolicyImpl.java:471)
           at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.newEnsemble(RackawareEnsemblePlacementPolicy.java:92)
           at org.apache.bookkeeper.client.BookieWatcher.newEnsemble(BookieWatcher.java:209)
           at org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:138)
           at org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:879)
           at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.initializeBookKeeper(ManagedLedgerImpl.java:366)
           at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.access$300(ManagedLedgerImpl.java:113)
           at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$1.operationComplete(ManagedLedgerImpl.java:305)
           at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$1.operationComplete(ManagedLedgerImpl.java:258)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper.lambda$null$0(MetaStoreImplZookeeper.java:150)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper$$Lambda$354/2112358463.processResult(Unknown
Source)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper.lambda$null$18(MetaStoreImplZookeeper.java:378)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper$$Lambda$357/778283669.accept(Unknown
Source)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper.lambda$asyncCreateFullPathOptimistic$21(MetaStoreImplZookeeper.java:382)
           at org.apache.bookkeeper.mledger.impl.MetaStoreImplZookeeper$$Lambda$355/1068452416.processResult(Unknown
Source)
           at org.apache.bookkeeper.zookeeper.ZooKeeperClient$10$1.processResult(ZooKeeperClient.java:737)
           at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:607)
           at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:505)
   ```
   
   Whenever the broker is shut down by reboot of bookie machine, this thread is waiting for
data to be returned from the ZK cache on the following line:
   https://github.com/apache/pulsar/blob/v2.1.1-incubating/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMapping.java#L153
   
   As far as I see the debug log, the following part is also processed by the same thread,
so `pulsar-ordered-OrderedExecutor-x-x-EventThread` seems to continue waiting for completion
of its own processing.
   https://github.com/apache/pulsar/blob/v2.1.1-incubating/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperCache.java#L322-L338
   
   The deadlock as described above occurs when the following warning log is output and a bookie
is quarantined:
   
   > 13:17:21.063 [BookKeeperClientScheduler-OrderedScheduler-0-0] WARN o.a.bookkeeper.client.BookieWatcher
- Bookie xxx.xxx.xxx.xxx:3181 has been quarantined because of read/write errors.
   
   If the list of quarantined bookies is not empty, the processing in the following loop which
is normally skipped is executed:
   https://github.com/apache/bookkeeper/blob/release-4.7.2/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L449-L455
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message