-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
Bookie down causes deadlock in broker #4635
Comments
@massakam
Also, I am thinking why thread was blocked at ZooKeeperDataCache::95 because we already have timeout on that blocking call. |
However,
In Is this similar to the part that caused the previous bug? |
It seems that this phenomenon doesn’t happen in following situations.
In our environment, rackawarePolicy is not set, but it is enabled. |
broker crash after one bookie down
|
@hrsakai Have you found and fixed the issue? @HeChuanXUPT from the log, it seems that the broker is not able to connect to the zookeeper. |
@sijie |
@hrsakai I see. thank you for your update! |
…eadlock and blocking IO on ZK thread (#8304) ### Motivation We have been seeing broker restarts due to zk-session timeout and that's because of #4635 and ``` "pulsar-ordered-OrderedExecutor-4-0-EventThread" #33 daemon prio=5 os_prio=0 cpu=36314.97ms elapsed=698.44s tid=0x00007f8114029790 nid=0x2a31 waiting on condition [0x00007f8170575000] java.lang.Thread.State: TIMED_WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.6/Native Method) - parking to wait for <0x00001001a1c1aa50> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.6/LockSupport.java:234) at java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.6/CompletableFuture.java:1798) at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.6/ForkJoinPool.java:3128) at java.util.concurrent.CompletableFuture.timedGet(java.base@11.0.6/CompletableFuture.java:1868) at java.util.concurrent.CompletableFuture.get(java.base@11.0.6/CompletableFuture.java:2021) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:293) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:238) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.getBlacklistedBookies(ZkIsolatedBookieEnsemblePlacementPolicy.java:150) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.newEnsemble(ZkIsolatedBookieEnsemblePlacementPolicy.java:123) 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:3063) at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1378) - locked <0x0000100035d21d60> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl) at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:188) at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:556) at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$935/0x00007f7f30252908.accept(Unknown Source) ``` The main reason of such zk-session timeout and broker-restart is all zk-event threads get blocked on ZK-Cache, [zk-session-wather](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperSessionWatcher.java) can't complete keep-alive and eventually zk-session timeout. Zk-thread gets blocked on `ZooKeeperCache` because [zk-cache](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperCache.java#L111) expires cache-entry at every 5 mins and Zk-thread misses the cache and tries to retrieve from zk. once, all zk-thread are blocked on Zk-Cache to get data from zk, it creates a deadlock and broker sees zk-session timeout. One of the solutions is to keep entries always available in cache, avoid cache miss and refresh zk-cache in background. this solution will make sure that zk-thread will not see cache-miss and will not be blocked. ### Modification - Refresh Zk-Cache in background without invalidating cache data so, it avoids zk-cache once entry is loaded at first time. ### Result It wil fix #4635
…eadlock and blocking IO on ZK thread (apache#8304) ### Motivation We have been seeing broker restarts due to zk-session timeout and that's because of apache#4635 and ``` "pulsar-ordered-OrderedExecutor-4-0-EventThread" apache#33 daemon prio=5 os_prio=0 cpu=36314.97ms elapsed=698.44s tid=0x00007f8114029790 nid=0x2a31 waiting on condition [0x00007f8170575000] java.lang.Thread.State: TIMED_WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.6/Native Method) - parking to wait for <0x00001001a1c1aa50> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.6/LockSupport.java:234) at java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.6/CompletableFuture.java:1798) at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.6/ForkJoinPool.java:3128) at java.util.concurrent.CompletableFuture.timedGet(java.base@11.0.6/CompletableFuture.java:1868) at java.util.concurrent.CompletableFuture.get(java.base@11.0.6/CompletableFuture.java:2021) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:293) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:238) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.getBlacklistedBookies(ZkIsolatedBookieEnsemblePlacementPolicy.java:150) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.newEnsemble(ZkIsolatedBookieEnsemblePlacementPolicy.java:123) 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:3063) at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1378) - locked <0x0000100035d21d60> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl) at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:188) at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:556) at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$935/0x00007f7f30252908.accept(Unknown Source) ``` The main reason of such zk-session timeout and broker-restart is all zk-event threads get blocked on ZK-Cache, [zk-session-wather](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperSessionWatcher.java) can't complete keep-alive and eventually zk-session timeout. Zk-thread gets blocked on `ZooKeeperCache` because [zk-cache](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperCache.java#L111) expires cache-entry at every 5 mins and Zk-thread misses the cache and tries to retrieve from zk. once, all zk-thread are blocked on Zk-Cache to get data from zk, it creates a deadlock and broker sees zk-session timeout. One of the solutions is to keep entries always available in cache, avoid cache miss and refresh zk-cache in background. this solution will make sure that zk-thread will not see cache-miss and will not be blocked. ### Modification - Refresh Zk-Cache in background without invalidating cache data so, it avoids zk-cache once entry is loaded at first time. ### Result It wil fix apache#4635
…eadlock and blocking IO on ZK thread (apache#8304) ### Motivation We have been seeing broker restarts due to zk-session timeout and that's because of apache#4635 and ``` "pulsar-ordered-OrderedExecutor-4-0-EventThread" apache#33 daemon prio=5 os_prio=0 cpu=36314.97ms elapsed=698.44s tid=0x00007f8114029790 nid=0x2a31 waiting on condition [0x00007f8170575000] java.lang.Thread.State: TIMED_WAITING (parking) at jdk.internal.misc.Unsafe.park(java.base@11.0.6/Native Method) - parking to wait for <0x00001001a1c1aa50> (a java.util.concurrent.CompletableFuture$Signaller) at java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.6/LockSupport.java:234) at java.util.concurrent.CompletableFuture$Signaller.block(java.base@11.0.6/CompletableFuture.java:1798) at java.util.concurrent.ForkJoinPool.managedBlock(java.base@11.0.6/ForkJoinPool.java:3128) at java.util.concurrent.CompletableFuture.timedGet(java.base@11.0.6/CompletableFuture.java:1868) at java.util.concurrent.CompletableFuture.get(java.base@11.0.6/CompletableFuture.java:2021) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:293) at org.apache.pulsar.zookeeper.ZooKeeperCache.getData(ZooKeeperCache.java:238) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.getBlacklistedBookies(ZkIsolatedBookieEnsemblePlacementPolicy.java:150) at org.apache.pulsar.zookeeper.ZkIsolatedBookieEnsemblePlacementPolicy.newEnsemble(ZkIsolatedBookieEnsemblePlacementPolicy.java:123) 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:3063) at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.ledgerClosed(ManagedLedgerImpl.java:1378) - locked <0x0000100035d21d60> (a org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl) at org.apache.bookkeeper.mledger.impl.OpAddEntry.closeComplete(OpAddEntry.java:188) at org.apache.bookkeeper.client.LedgerHandle$5.lambda$safeRun$0(LedgerHandle.java:556) at org.apache.bookkeeper.client.LedgerHandle$5$$Lambda$935/0x00007f7f30252908.accept(Unknown Source) ``` The main reason of such zk-session timeout and broker-restart is all zk-event threads get blocked on ZK-Cache, [zk-session-wather](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperSessionWatcher.java) can't complete keep-alive and eventually zk-session timeout. Zk-thread gets blocked on `ZooKeeperCache` because [zk-cache](https://github.com/apache/pulsar/blob/master/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperCache.java#L111) expires cache-entry at every 5 mins and Zk-thread misses the cache and tries to retrieve from zk. once, all zk-thread are blocked on Zk-Cache to get data from zk, it creates a deadlock and broker sees zk-session timeout. One of the solutions is to keep entries always available in cache, avoid cache miss and refresh zk-cache in background. this solution will make sure that zk-thread will not see cache-miss and will not be blocked. ### Modification - Refresh Zk-Cache in background without invalidating cache data so, it avoids zk-cache once entry is loaded at first time. ### Result It wil fix apache#4635
One of multiple bookie servers in our cluster went down due to a hardware failure. At the same time, the broker server went down. Messages that the broker could not connect to ZK were output to its log. I think this is due to a deadlock.
Below is a thread dump just before the broker shuts down.
broker_threaddump.txt
This phenomenon is similar to #3566. However the Pulsar version of the broker is 2.3.2, and the previous bug should have already been fixed.
The text was updated successfully, but these errors were encountered: