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

[pulsar-broker] Refresh ZooKeeper-data cache in background to avoid d… #8304

Merged
merged 1 commit into from
Oct 30, 2020

Conversation

rdhabalia
Copy link
Contributor

…eadlock and blocking IO on ZK thread

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 can't complete keep-alive and eventually zk-session timeout. Zk-thread gets blocked on ZooKeeperCache because zk-cache 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

@rdhabalia rdhabalia added this to the 2.7.0 milestone Oct 20, 2020
@rdhabalia rdhabalia self-assigned this Oct 20, 2020
@rdhabalia
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@rdhabalia
Copy link
Contributor Author

/pulsarbot run-failure-checks

@jiazhai
Copy link
Member

jiazhai commented Oct 20, 2020

/pulsarbot run-failure-checks

@rdhabalia
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@jiazhai
Copy link
Member

jiazhai commented Oct 21, 2020

/pulsarbot run-failure-checks

@rdhabalia rdhabalia force-pushed the zk_cache_sync branch 2 times, most recently from 2572dc6 to 1379947 Compare October 23, 2020 06:06
@rdhabalia
Copy link
Contributor Author

/pulsarbot run-failure-checks

1 similar comment
@rdhabalia
Copy link
Contributor Author

/pulsarbot run-failure-checks

@lhotari
Copy link
Member

lhotari commented Oct 30, 2020

FYI, there's another related fix #8406 that was already merged.

…eadlock and blocking IO on ZK thread

fix test
@codelipenghui
Copy link
Contributor

@wolfstudy After the CI passed, please help cherry-pick to 2.6.2, thanks.

@wolfstudy
Copy link
Member

@wolfstudy After the CI passed, please help cherry-pick to 2.6.2, thanks.

Sure, will process it.

@lhotari
Copy link
Member

lhotari commented Oct 30, 2020

/pulsarbot run-failure-checks

@wolfstudy wolfstudy merged commit bebce4f into apache:master Oct 30, 2020
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request 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 pull request 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
codelipenghui added a commit to codelipenghui/incubator-pulsar that referenced this pull request Nov 20, 2020
jiazhai pushed a commit that referenced this pull request Nov 20, 2020
Fix cherry-pick issue of the #8304
codelipenghui added a commit to streamnative/pulsar-archived that referenced this pull request Nov 20, 2020
Fix cherry-pick issue of the apache#8304

(cherry picked from commit 3795d1f)
codelipenghui added a commit to codelipenghui/incubator-pulsar that referenced this pull request Nov 21, 2020
codelipenghui added a commit to streamnative/pulsar-archived that referenced this pull request Nov 21, 2020
codelipenghui added a commit to streamnative/pulsar-archived that referenced this pull request Nov 22, 2020
apache#8652)

(cherry picked from commit 45125a1)
(cherry picked from commit 6cab86a)
codelipenghui added a commit to streamnative/pulsar-archived that referenced this pull request Nov 22, 2020
Fix cherry-pick issue of the apache#8304

(cherry picked from commit 3795d1f)
@rdhabalia rdhabalia deleted the zk_cache_sync branch January 11, 2021 00:20
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

Bookie down causes deadlock in broker
6 participants