Skip to content

Conversation

@massakam
Copy link
Contributor

@massakam massakam commented Jul 29, 2020

Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)

This is due to a deadlock occurring in the proxy. While this deadlock occurs, main-EventThread is stopped in the following states:

"main-EventThread" #26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)

ZookeeperCacheLoader will try to get load information for all available brokers from ZK or its cache if there is a change in the znode /loadbalance/brokers.

The problem is that it is main-EventThread that performs the above process, and it is also main-EventThread that gets the data from ZK.

main-EventThread is blocked at the following line, so deadlock occurs.

return getAsync(path).get(zkOperationTimeoutSeconds, TimeUnit.SECONDS);

Modifications

Make the method ZookeeperCacheLoader#updateBrokerList run on a thread separate from main-EventThread.

Made updateBrokerList() method asynchronous so that main-EventThread is not blocked.

@massakam massakam added type/bug The PR fixed a bug or issue reported a bug area/proxy release/2.6.2 labels Jul 29, 2020
@massakam massakam added this to the 2.7.0 milestone Jul 29, 2020
@massakam massakam self-assigned this Jul 29, 2020
@codelipenghui
Copy link
Contributor

/pulsarbot run-failure-checks

// Run in a separate thread to avoid deadlocks
brokerListUpdater.execute(() -> {
try {
updateBrokerList(brokerNodes);
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need a blocking call here? Why can't we just make this method async? So we don't need to introduce another thread.

Copy link
Contributor Author

@massakam massakam Jul 30, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think we need a blocking call here. Made the updateBrokerList() method asynchronous.

@massakam
Copy link
Contributor Author

There was similar code in pulsar-discovery-service, so I fixed that as well.

@massakam
Copy link
Contributor Author

@sijie PTAL

@massakam
Copy link
Contributor Author

massakam commented Aug 5, 2020

@sijie Could you review this again?

@massakam massakam requested a review from sijie August 6, 2020 01:56
@sijie sijie merged commit d7c1cfa into apache:master Aug 6, 2020
@massakam massakam deleted the fix-proxy-deadlock branch August 6, 2020 15:43
cdbartholomew pushed a commit to kafkaesque-io/pulsar that referenced this pull request Aug 15, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" apache#26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.
huangdx0726 pushed a commit to huangdx0726/pulsar that referenced this pull request Aug 24, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" apache#26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" apache#26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" apache#26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.
lbenc135 pushed a commit to lbenc135/pulsar that referenced this pull request Sep 5, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" apache#26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.
wolfstudy pushed a commit that referenced this pull request Oct 30, 2020
### Motivation

When a broker server is restarted, the Pulsar proxy outputs the following warning:

```
15:09:01.486 [main-EventThread] INFO  o.a.p.z.ZooKeeperChildrenCache       - reloadCache called in zookeeperChildrenCache for path /loadbalance/brokers
15:09:31.487 [main-EventThread] WARN  o.a.p.p.s.util.ZookeeperCacheLoader  - Error updating broker info after broker list changed.
java.util.concurrent.TimeoutException: null
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1784)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:118)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:82)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

This is due to a deadlock occurring in the proxy. While this deadlock occurs, `main-EventThread` is stopped in the following states:
```
"main-EventThread" #26 daemon prio=5 os_prio=0 tid=0x00007fa9b55ff000 nid=0x3d98 waiting on condition [0x00007fa923dfc000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000eef2ad80> (a java.util.concurrent.CompletableFuture$Signaller)
        at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
        at java.util.concurrent.CompletableFuture$Signaller.block(CompletableFuture.java:1709)
        at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
        at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1788)
        at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1928)
        at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.updateBrokerList(ZookeeperCacheLoader.java:123)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader.lambda$new$0(ZookeeperCacheLoader.java:84)
        at org.apache.pulsar.proxy.server.util.ZookeeperCacheLoader$$Lambda$40/1450652220.onUpdate(Unknown Source)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache.lambda$0(ZooKeeperChildrenCache.java:89)
        at org.apache.pulsar.zookeeper.ZooKeeperChildrenCache$$Lambda$365/191748085.accept(Unknown Source)
        at java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:670)
        at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:646)
        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
        at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1975)
        at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$22(ZooKeeperCache.java:415)
        at org.apache.pulsar.zookeeper.ZooKeeperCache$$Lambda$46/1819738265.processResult(Unknown Source)
        at org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:592)
        at org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:508)
```

`ZookeeperCacheLoader` will try to get load information for all available brokers from ZK or its cache if there is a change in the znode `/loadbalance/brokers`.

The problem is that it is `main-EventThread` that performs the above process, and it is also `main-EventThread` that gets the data from ZK.

`main-EventThread` is blocked at the following line, so deadlock occurs.
https://github.com/apache/pulsar/blob/9d3ab60efddd3f4064789293fd8e9c4d4388b6c7/pulsar-zookeeper-utils/src/main/java/org/apache/pulsar/zookeeper/ZooKeeperDataCache.java#L97

### Modifications

~Make the method `ZookeeperCacheLoader#updateBrokerList` run on a thread separate from `main-EventThread`.~

Made `updateBrokerList()` method asynchronous so that `main-EventThread` is not blocked.

(cherry picked from commit d7c1cfa)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

area/proxy release/2.6.2 type/bug The PR fixed a bug or issue reported a bug

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants