sandeep-mst opened a new issue, #25058: URL: https://github.com/apache/pulsar/issues/25058
### Search before reporting - [x] I searched in the [issues](https://github.com/apache/pulsar/issues) and found nothing similar. ### Read release policy - [x] I understand that [unsupported versions](https://pulsar.apache.org/contribute/release-policy/#supported-versions) don't get bug fixes. I will attempt to reproduce the issue on a supported version of Pulsar client and Pulsar broker. ### User environment - master - java 17 ### Issue Description WatchTask of Zookeeper [writableBookies](https://github.com/apache/bookkeeper/blob/2c5d98ebbe1ff8e935c4efd29aa92f37341452ec/bookkeeper-server/src/main/java/org/apache/bookkeeper/discover/ZKRegistrationClient.java#L383) has two listeners related to rackAwareness attached to it. 1. [ZkBookieRackAffinityMapping](https://github.com/apache/pulsar/blob/82237d3684fe506bcb6426b3b23f413422e6e4fb/pulsar-broker-common/src/main/java/org/apache/pulsar/zookeeper/ZkBookieRackAffinityMapping.java) which extends [BookieRackAffinityMapping](https://github.com/apache/pulsar/blob/efa5e8b04018356447ec1744c6e083430e8e1f05/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java) and responsible for maintaining the rackInfoMap and racksWithHost which is used by placement policies to get the updated rackInfo. 2. [RackawareEnsemblePlacementPolicy](https://github.com/apache/bookkeeper/blob/fc981ba04bae126afe3452b76006e80487cc9d84/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicy.java) which uses the ZkBookieRackAffinityMapping to resolve the network location. As they are both listeners without any ordering, there is a real chance where the [resolve network location(rack path)](https://github.com/apache/pulsar/blob/efa5e8b04018356447ec1744c6e083430e8e1f05/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java#L212) is called by RackAwarePlacementPolicy before the [updateRacksWithHost](https://github.com/apache/pulsar/blob/efa5e8b04018356447ec1744c6e083430e8e1f05/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java#L172) is called on the BookieRackAffinityMapping which refreshes the rack information. This would lead to the placement policy unable to update find the network location (rack path) of the bookie and fall back into default-rack. This will get updated when the next watchEvent is triggered. At the moment, rackawarePolicy.onBookieRackChange(new ArrayList<>(bookieIdSet)) is called only when the bookie rack info changes on the zookeeper and not when updateRacksWithHost is called. Updating the code to call onBookieRackChange when the [updateRacksWithHost](https://github.com/apache/pulsar/blob/efa5e8b04018356447ec1744c6e083430e8e1f05/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java#L172) completed should fix this issue. ### Error messages ```text org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to resolve network location for XXX.XXX.XXX.XXX, using default rack for it : /default-rack. ``` ### Reproducing the issue Configure RackAwarenessPlacementPolicy with the following config and bring down a bookie. > ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy reppDnsResolverClass=org.apache.pulsar.zookeeper.ZkBookieRackAffinityMapping enforceMinNumRacksPerWriteQuorum=true minNumRacksPerWriteQuorum=3 bookkeeperMetadataServiceUri=zk+heirarchical://url/ledgers ### Additional information _No response_ ### Are you willing to submit a PR? - [x] I'm willing to submit a PR! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
