apache / pulsar

Apache Pulsar - distributed pub-sub messaging system
https://pulsar.apache.org/
Apache License 2.0
14.22k stars 3.58k forks source link

Investigate root cause for quarantined test RackAwareTest.testPlacement since it might be a real production code issue #11433

Open lhotari opened 3 years ago

lhotari commented 3 years ago

Describe the bug

Flaky test RackawareTest.testPlacement has been moved to quarantine test group so that it doesn't block CI, this change was made in #11370.

The root cause seems to be a production code issue. The stacktrace shown in the logs of a failing test hints that there might be a dead lock that happens. One common issue is locking up Zookeeper threads when Zookeeper operations are initiated from a zookeeper thread that notifies about a change (example of such issue is #10418).

12:15:18.019 [ForkJoinPool.commonPool-worker-1] WARN  org.apache.pulsar.zookeeper.ZooKeeperDataCache - Reloading ZooKeeperDataCache failed at path: /bookies
java.util.concurrent.CompletionException: java.lang.RuntimeException: java.util.concurrent.TimeoutException
    at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:314) ~[?:?]
    at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:319) ~[?:?]
    at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:718) ~[?:?]
    at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
    at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) ~[?:?]
    at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$getDataAsync$17(ZooKeeperCache.java:386) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714) ~[?:?]
    at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) ~[?:?]
    at java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) ~[?:?]
    at org.apache.pulsar.zookeeper.ZooKeeperCache.lambda$getDataAsync$11(ZooKeeperCache.java:366) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at java.util.concurrent.ForkJoinTask$RunnableExecuteAction.exec(ForkJoinTask.java:1426) [?:?]
    at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:290) [?:?]
    at java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(ForkJoinPool.java:1020) [?:?]
    at java.util.concurrent.ForkJoinPool.scan(ForkJoinPool.java:1656) [?:?]
    at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1594) [?:?]
    at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:183) [?:?]
Caused by: java.lang.RuntimeException: java.util.concurrent.TimeoutException
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.getRack(ZkBookieRackAffinityMapping.java:195) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.resolve(ZkBookieRackAffinityMapping.java:179) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy$DNSResolverDecorator.resolve(TopologyAwareEnsemblePlacementPolicy.java:561) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.net.NetUtils.resolveNetworkLocation(NetUtils.java:88) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.resolveNetworkLocation(TopologyAwareEnsemblePlacementPolicy.java:794) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.createBookieNode(TopologyAwareEnsemblePlacementPolicy.java:784) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.onBookieRackChange(TopologyAwareEnsemblePlacementPolicy.java:747) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.onBookieRackChange(RackawareEnsemblePlacementPolicyImpl.java:80) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.onUpdate(ZkBookieRackAffinityMapping.java:242) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.onUpdate(ZkBookieRackAffinityMapping.java:53) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZooKeeperDataCache.lambda$reloadCache$3(ZooKeeperDataCache.java:138) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714) ~[?:?]
    ... 13 more
Caused by: java.util.concurrent.TimeoutException
    at java.util.concurrent.CompletableFuture.timedGet(CompletableFuture.java:1886) ~[?:?]
    at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2021) ~[?:?]
    at org.apache.pulsar.zookeeper.ZooKeeperDataCache.get(ZooKeeperDataCache.java:97) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.getRack(ZkBookieRackAffinityMapping.java:187) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.resolve(ZkBookieRackAffinityMapping.java:179) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy$DNSResolverDecorator.resolve(TopologyAwareEnsemblePlacementPolicy.java:561) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.net.NetUtils.resolveNetworkLocation(NetUtils.java:88) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.resolveNetworkLocation(TopologyAwareEnsemblePlacementPolicy.java:794) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.createBookieNode(TopologyAwareEnsemblePlacementPolicy.java:784) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.onBookieRackChange(TopologyAwareEnsemblePlacementPolicy.java:747) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.onBookieRackChange(RackawareEnsemblePlacementPolicyImpl.java:80) ~[bookkeeper-server-4.14.1.jar:4.14.1]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.onUpdate(ZkBookieRackAffinityMapping.java:242) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping.onUpdate(ZkBookieRackAffinityMapping.java:53) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at org.apache.pulsar.zookeeper.ZooKeeperDataCache.lambda$reloadCache$3(ZooKeeperDataCache.java:138) ~[pulsar-zookeeper-utils-2.9.0-SNAPSHOT.jar:2.9.0-SNAPSHOT]
    at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714) ~[?:?]
    ... 13 more
lhotari commented 2 years ago

There's also #14106 about RackAwareTest.testPlacement

TakaHiR07 commented 2 years ago

I have encounter the same problem when do cluster expansion in our production environment.

We firstly change rack information in zk /bookies, and then add broker and bookie to our cluster. Then the same Timeout Exception occur. I guess it has successfully get data from /bookies, but block due to dead lock in executors.