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

Bookie down causes deadlock in broker #4635

Closed
massakam opened this issue Jun 28, 2019 · 7 comments · Fixed by #8304
Closed

Bookie down causes deadlock in broker #4635

massakam opened this issue Jun 28, 2019 · 7 comments · Fixed by #8304
Labels
type/bug The PR fixed a bug or issue reported a bug

Comments

@massakam
Copy link
Contributor

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.

19:38:55.846 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 25 seconds
19:38:57.846 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 23 seconds
19:38:59.847 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 21 seconds
19:39:01.847 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 19 seconds
19:39:03.847 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 16 seconds
19:39:05.847 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 14 seconds
19:39:07.848 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 12 seconds
19:39:09.848 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 10 seconds
19:39:11.848 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 8 seconds
19:39:13.849 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 6 seconds
19:39:15.849 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 4 seconds
19:39:17.849 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 2 seconds
19:39:19.849 [pulsar-zk-session-watcher-5-1] WARN  o.a.p.z.ZooKeeperSessionWatcher      - zoo keeper disconnected, waiting to reconnect, time remaining = 0 seconds
19:39:21.850 [pulsar-zk-session-watcher-5-1] ERROR o.a.p.z.ZooKeeperSessionWatcher      - timeout expired for reconnecting, invoking shutdown service

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.

@massakam massakam added the type/bug The PR fixed a bug or issue reported a bug label Jun 28, 2019
@rdhabalia
Copy link
Contributor

@massakam
Thread-dump shows that most of the bk-client-ordered and pulsar-ordered threads are waiting on
at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(.java:95) at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.getRack(ZkBookieRackAffinityMapping.java:154)
and I think thread should be unblocked at that place due to this fix: #3633 addressed in 2.3.1. and it seems thread-dump doesn't show clear evidence that broker went down because one of the bookie shutdown.

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(

.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)

Also, I am thinking why thread was blocked at ZooKeeperDataCache::95 because we already have timeout on that blocking call.
https://github.com/apache/pulsar/blob/v2.3.1/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L95

@massakam
Copy link
Contributor Author

massakam commented Jul 3, 2019

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

@hrsakai
Copy link
Contributor

hrsakai commented Jul 11, 2019

It seems that this phenomenon doesn’t happen in following situations.

  • pulsar-broker-2.2.1 (the bk version of brokers is 4.7.3)
  • pulsar-broker-2.3.2 (the bk version of brokers is 4.9.2) and bookkeeperClientRackawarePolicyEnabled=false

In our environment, rackawarePolicy is not set, but it is enabled.

@HeChuanXUPT
Copy link

broker crash after one bookie down
version: 2.5.0

06:57:57.231 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 25 seconds
06:57:59.231 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 23 seconds
06:58:01.231 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 21 seconds
06:58:03.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 19 seconds
06:58:05.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 17 seconds
06:58:07.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 15 seconds
06:58:09.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 13 seconds
06:58:11.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 11 seconds
06:58:13.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 8 seconds
06:58:15.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 6 seconds
06:58:17.232 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 4 seconds
06:58:19.233 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 2 seconds
06:58:21.233 [pulsar-zk-session-watcher-5-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - zoo keeper disconnected, waiting to reconnect, time remaining = 0 seconds
06:58:23.233 [pulsar-zk-session-watcher-5-1] ERROR org.apache.pulsar.zookeeper.ZooKeeperSessionWatcher - timeout expired for reconnecting, invoking shutdown service

@sijie
Copy link
Member

sijie commented Jul 6, 2020

@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.

@hrsakai
Copy link
Contributor

hrsakai commented Jul 6, 2020

@sijie
I haven't. In our clusters, we turn off bookkeeperClientRackawarePolicy to avoid the issue.

@sijie
Copy link
Member

sijie commented Jul 6, 2020

@hrsakai I see. thank you for your update!

wolfstudy pushed a commit that referenced this issue Oct 30, 2020
…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
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this issue Nov 13, 2020
…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
flowchartsman pushed a commit to flowchartsman/pulsar that referenced this issue Nov 17, 2020
…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
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type/bug The PR fixed a bug or issue reported a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

5 participants