michaeljmarshall opened a new pull request, #16825:
URL: https://github.com/apache/pulsar/pull/16825

   ### Motivation
   
   The `BookieRackAffinityMapping` class relies on a metadata cache that 
expires entries after 10 minutes. When an entry expires, the next call to 
`BookieRackAffinityMapping#getRack` returns `null` (because the entry expired) 
and the `TopologyAwareEnsemblePlacementPolicy` (bookkeeper class) stores the 
bookie's network location as `default-rack`.
   
   It is trivial to reproduce the issue. Start a Pulsar cluster, define a rack 
topology, wait for at least 10 minutes, kill one of the bookies that is not in 
the default-rack, and observe the broker logs as the bookie comes back. The 
broker will log that the bookie is a member of the default-rack. When 
`bookkeeperClientEnforceMinNumRacksPerWriteQuorum` is enabled in the broker, 
this bug becomes a blocking issue where the only way to resolve the bad state 
is to restart the broker (or to restart the bookie assuming the broker still 
has the right mapping in the cache).
   
   This PR changes the design of the `BookieRackAffinityMapping` by removing 
cache expiration. When the broker starts up, it will discover the mapping from 
zookeeper and store that mapping until the broker observes an update from a ZK 
watch.
   
   ### Modifications
   
   * Rely on an indefinitely cached rack mapping in 
`BookieRackAffinityMapping`, instead of relying on a metadata cache, which is 
defined to have an entry expiration.
   * Eagerly resolve the bookie mapping. This was removed in 
https://github.com/apache/pulsar/pull/12097, but now that 
https://github.com/apache/bookkeeper/pull/2788 is merged and available in the 
bookkeeper client, we can safely resolve the addresses early.
   * Add `synchronized` keyword to all relevant methods that modify mutable 
state from multiple threads. Based on my reading of the code, there is not a 
risk for deadlock with this change. Making these methods synchronized also 
prevents certain races that could negatively affect bookie network location 
resolution. The only potential problem is that this synchronization could block 
a zk callback thread briefly. Because the operations in these methods do not 
contain any blocking io (other than on initialization), I view blocking a zk 
thread as unlikely.
   * Remove the `volatile` keyword for two maps that are now only updated 
within `synchronized` blocks.
   * Move the `registerListener` call to before getting the value from 
zookeeper. This ensures that an update is not missed in the very short time 
between getting the value and registering the listener. Because the method is 
synchronized, the event will properly be observed after the original 
initialization.
   * Update a test to use Awaitility to account for the asynchronous nature of 
metadata store notifications.
   * Move the `rackawarePolicy` null check to later in the sequence to make 
tests pass. Note that we always use a `rackawarePolicy`, so this is a trivial 
change.
   
   ### Verifying this change
   
   This change is covered by existing tests. Note that the original bug is 
challenging to reproduce in a unit test because the bug relies on cache 
expiration, which is hard coded at 10 minutes in the `MetadataCacheImpl`. By 
removing any chance for cache expiration, we remove the possibility for this 
bug.
   
   ### Additional Context
   
   Here are sample logs from a reproduction of the issue:
   
   ```
   2022-07-27T15:20:55,352+0000 [main-EventThread] INFO  
org.apache.bookkeeper.net.NetworkTopologyImpl - Removing a node: 
/az1/pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local:3181
   2022-07-27T15:20:55,353+0000 [main-EventThread] INFO  
org.apache.bookkeeper.net.NetworkTopologyImpl - Removing a node: 
/az1/pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local:3181
   2022-07-27T15:20:59,310+0000 [main-EventThread] WARN  
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to 
resolve network location for 
pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local, using 
default rack for it : /default-rack.
   2022-07-27T15:20:59,310+0000 [main-EventThread] INFO  
org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: 
/default-rack/pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local:3181
   2022-07-27T15:20:59,311+0000 [main-EventThread] WARN  
org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy - Failed to 
resolve network location for 
pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local, using 
default rack for it : /default-rack.
   2022-07-27T15:20:59,311+0000 [main-EventThread] INFO  
org.apache.bookkeeper.net.NetworkTopologyImpl - Adding a new node: 
/default-rack/pulsar-bookkeeper-3.pulsar-bookkeeper.michael-test.svc.cluster.local:3181
   ```
   
   ### Alternative Solution
   
   An alternative solution is to add a callback to the metadata store's result 
when the future is not complete. The callback would trigger the logic in the 
`BookieRackAffinityMapping#handleUpdates`. While this change would be smaller 
in terms of lines of code touched, I view it as suboptimal because it 
necessarily leads to misclassification of bookies as members of the 
`default-rack`, which is both confusing to users and could lead to temporary 
errors.
   
   ### Does this pull request potentially affect one of the following parts:
   
   This PR does not introduce any breaking changes. It might not easily get 
cherry picked to older release branches.
   
   ### Documentation
   
   - [x] `doc-not-needed` 
   
   Docs are not needed because this is just an internal bug fix.


-- 
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]

Reply via email to