This is an automated email from the ASF dual-hosted git repository.

penghui pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/master by this push:
     new d9ebaf5bf6f [fix][broker] rackaware policy is ineffective when delete 
zk rack info after bkclient initialize (#20944)
d9ebaf5bf6f is described below

commit d9ebaf5bf6fda44d21ac24cec7dbe208b59dc597
Author: ken <[email protected]>
AuthorDate: Sat Oct 7 20:24:45 2023 +0800

    [fix][broker] rackaware policy is ineffective when delete zk rack info 
after bkclient initialize (#20944)
---
 .../bookie/rackawareness/BookieRackAffinityMapping.java |  4 ++--
 .../rackawareness/BookieRackAffinityMappingTest.java    | 17 +++++++++++++++++
 2 files changed, 19 insertions(+), 2 deletions(-)

diff --git 
a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java
 
b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java
index e9e350800b4..d54ef2a5f4c 100644
--- 
a/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java
+++ 
b/pulsar-broker-common/src/main/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMapping.java
@@ -121,8 +121,6 @@ public class BookieRackAffinityMapping extends 
AbstractDNSToSwitchMapping
             store.registerListener(this::handleUpdates);
             racksWithHost = bookieMappingCache.get(BOOKIE_INFO_ROOT_PATH).get()
                     .orElseGet(BookiesRackConfiguration::new);
-            updateRacksWithHost(racksWithHost);
-            watchAvailableBookies();
             for (Map<String, BookieInfo> bookieMapping : 
racksWithHost.values()) {
                 for (String address : bookieMapping.keySet()) {
                     bookieAddressListLastTime.add(BookieId.parse(address));
@@ -132,6 +130,8 @@ public class BookieRackAffinityMapping extends 
AbstractDNSToSwitchMapping
                             bookieAddressListLastTime);
                 }
             }
+            updateRacksWithHost(racksWithHost);
+            watchAvailableBookies();
         } catch (InterruptedException | ExecutionException | MetadataException 
e) {
             throw new RuntimeException(METADATA_STORE_INSTANCE + " failed to 
init BookieId list");
         }
diff --git 
a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java
 
b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java
index d7be7dabd0d..d7df5afb4be 100644
--- 
a/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java
+++ 
b/pulsar-broker-common/src/test/java/org/apache/pulsar/bookie/rackawareness/BookieRackAffinityMappingTest.java
@@ -254,6 +254,7 @@ public class BookieRackAffinityMappingTest {
                 bkClientConf.getTimeoutTimerNumTicks());
 
         RackawareEnsemblePlacementPolicy repp = new 
RackawareEnsemblePlacementPolicy();
+        mapping.registerRackChangeListener(repp);
         Class<?> clazz1 = 
Class.forName("org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy");
         Field field1 = clazz1.getDeclaredField("knownBookies");
         field1.setAccessible(true);
@@ -323,6 +324,22 @@ public class BookieRackAffinityMappingTest {
         
assertEquals(knownBookies.get(BOOKIE2.toBookieId()).getNetworkLocation(), 
"/rack1");
         
assertEquals(knownBookies.get(BOOKIE3.toBookieId()).getNetworkLocation(), 
"/default-rack");
 
+        //remove bookie2 rack, the bookie2 rack should be /default-rack
+        data = "{\"group1\": {\"" + BOOKIE1
+                + "\": {\"rack\": \"/rack0\", \"hostname\": 
\"bookie1.example.com\"}}}";
+        store.put(BookieRackAffinityMapping.BOOKIE_INFO_ROOT_PATH, 
data.getBytes(), Optional.empty()).join();
+        Awaitility.await().atMost(30, TimeUnit.SECONDS).until(() -> 
((BookiesRackConfiguration)field.get(mapping)).get("group1").size() == 1);
+
+        racks = mapping
+                .resolve(Lists.newArrayList(BOOKIE1.getHostName(), 
BOOKIE2.getHostName(), BOOKIE3.getHostName()))
+                .stream().filter(Objects::nonNull).toList();
+        assertEquals(racks.size(), 1);
+        assertEquals(racks.get(0), "/rack0");
+        assertEquals(knownBookies.size(), 3);
+        
assertEquals(knownBookies.get(BOOKIE1.toBookieId()).getNetworkLocation(), 
"/rack0");
+        
assertEquals(knownBookies.get(BOOKIE2.toBookieId()).getNetworkLocation(), 
"/default-rack");
+        
assertEquals(knownBookies.get(BOOKIE3.toBookieId()).getNetworkLocation(), 
"/default-rack");
+
         timer.stop();
     }
 }

Reply via email to