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

chenhang pushed a commit to branch branch-4.17
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git

commit c4215c6f55bd90af056e5a8a37283f38029c0ff6
Author: Hang Chen <[email protected]>
AuthorDate: Sat May 11 20:39:44 2024 +0800

    Fix disk weight ensemble infinite loop bug (#4324)
    
    ### Motivation
    
    In the RackawareEnsemblePlacementPolicyImpl#selectRandomInternal method, if 
bookieNode in the ensemble list is not added to the excludeBookies set and the 
diskWeight feature is enabled, this method will fall into an infinite loop.
    
https://github.com/apache/bookkeeper/blob/3ed93a0342ccff8c30ea472d731a7b593b4d32b0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L773-L777
    
    The root cause is the newly selected bookies already exist in the ensemble 
list, leading the `ensemble.addNode(bookie)` method to return `false` and the 
while loop continues.
    
https://github.com/apache/bookkeeper/blob/3ed93a0342ccff8c30ea472d731a7b593b4d32b0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L841
    
    In the next round of the loop, `bookiesSeenSoFar.size()` will never equal 
to `bookiesToSelectFrom.size()` due to the wRSelection set doesn't exclude the 
bookies in the ensemble set.
    
https://github.com/apache/bookkeeper/blob/3ed93a0342ccff8c30ea472d731a7b593b4d32b0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L815-L822
    
    ### Changes
    Remove the following code to make sure the items in bookiesToSelectFrom are 
all included in `wRSelection`
    
https://github.com/apache/bookkeeper/blob/3ed93a0342ccff8c30ea472d731a7b593b4d32b0/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L795-L797
    
    (cherry picked from commit a51f94dbf015c69efad9fcc4b99a88efda443e3c)
---
 .../RackawareEnsemblePlacementPolicyImpl.java      |  3 --
 .../TestRackawareEnsemblePlacementPolicy.java      | 57 ++++++++++++++++++++--
 2 files changed, 54 insertions(+), 6 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
index 3863a26a24..dba9828e85 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java
@@ -792,9 +792,6 @@ public class RackawareEnsemblePlacementPolicyImpl extends 
TopologyAwareEnsembleP
 
             Map<BookieNode, WeightedObject> rackMap = new HashMap<BookieNode, 
WeightedObject>();
             for (BookieNode n : bookiesToSelectFrom) {
-                if (excludeBookies.contains(n)) {
-                    continue;
-                }
                 if (this.bookieInfoMap.containsKey(n)) {
                     rackMap.put(n, this.bookieInfoMap.get(n));
                 } else {
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java
index ed37159ee1..ac77374706 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicy.java
@@ -17,6 +17,7 @@
  */
 package org.apache.bookkeeper.client;
 
+import static 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.RACKNAME_DISTANCE_FROM_LEAVES;
 import static 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.REPP_DNS_RESOLVER_CLASS;
 import static 
org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.shuffleWithMask;
 import static 
org.apache.bookkeeper.client.RoundRobinDistributionSchedule.writeSetFromValues;
@@ -790,6 +791,55 @@ public class TestRackawareEnsemblePlacementPolicy extends 
TestCase {
         }
     }
 
+    @Test(timeout = 30_000)
+    public void testNewEnsembleWithExcludeBookies() throws Exception {
+        repp.uninitalize();
+        updateMyRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
+
+        repp = new RackawareEnsemblePlacementPolicy();
+        conf.setDiskWeightBasedPlacementEnabled(true);
+        repp.initialize(conf, Optional.<DNSToSwitchMapping>empty(), timer,
+                DISABLE_ALL, NullStatsLogger.INSTANCE, 
BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER);
+        repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
+
+        BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.2", 3181);
+        BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.3", 3181);
+        BookieSocketAddress addr3 = new BookieSocketAddress("127.0.0.4", 3181);
+
+        BookieNode addr1Node = new BookieNode(addr1.toBookieId(), 
repp.resolveNetworkLocation(addr1.toBookieId()));
+        BookieNode addr2Node = new BookieNode(addr2.toBookieId(), 
repp.resolveNetworkLocation(addr2.toBookieId()));
+        BookieNode addr3Node = new BookieNode(addr3.toBookieId(), 
repp.resolveNetworkLocation(addr3.toBookieId()));
+        // update dns mapping
+        StaticDNSResolver.addNodeToRack(addr1.getHostName(), 
"/default-region/r1");
+        StaticDNSResolver.addNodeToRack(addr2.getHostName(), 
"/default-region/r2");
+        StaticDNSResolver.addNodeToRack(addr3.getHostName(), 
"/default-region/r1");
+        // Update cluster
+        Set<BookieId> addrs = new HashSet<BookieId>();
+        addrs.add(addr1.toBookieId());
+        addrs.add(addr2.toBookieId());
+        addrs.add(addr3.toBookieId());
+        repp.onClusterChanged(addrs, new HashSet<BookieId>());
+
+        Set<Node> excludeBookies = new HashSet<>();
+        excludeBookies.add(addr2Node);
+        excludeBookies.add(addr3Node);
+
+        TopologyAwareEnsemblePlacementPolicy.RRTopologyAwareCoverageEnsemble 
ensemble =
+                new 
TopologyAwareEnsemblePlacementPolicy.RRTopologyAwareCoverageEnsemble(
+                        2, 2, 2,
+                        RACKNAME_DISTANCE_FROM_LEAVES,
+                        null, null, 1);
+        ensemble.addNode(new BookieNode(addr1.toBookieId(), 
repp.resolveNetworkLocation(addr1.toBookieId())));
+        try {
+           repp.selectRandomInternal(null, 1, excludeBookies, null, ensemble);
+           fail("Should fail with not enough bookies exception");
+        } catch (BKNotEnoughBookiesException ex) {
+            //
+        }
+
+        conf.setDiskWeightBasedPlacementEnabled(false);
+    }
+
     @Test
     public void testSingleRackWithEnforceMinNumRacks() throws Exception {
         repp.uninitalize();
@@ -2061,10 +2111,11 @@ public class TestRackawareEnsemblePlacementPolicy 
extends TestCase {
             selectionCounts.put(replacedBookie, 
selectionCounts.get(replacedBookie) + 1);
         }
         /*
-         * since addr2 has to be replaced, the remaining bookies weight are - 
50, 100, 200, 500 (10*50)
-         * So the median calculated by WeightedRandomSelection is (100 + 200) 
/ 2 = 150
+         * Even though addr2 has to be replaced, but being excluded bookie 
weight is not excluded in the choose list.
+         * All the bookies weight are - 50, 100, 100, 200, 500 (10*50)
+         * So the median calculated by WeightedRandomSelection is 100
          */
-        double medianWeight = 150;
+        double medianWeight = 100;
         double medianSelectionCounts = (double) (medianWeight / 
bookieInfoMap.get(addr1.toBookieId()).getWeight())
             * selectionCounts.get(addr1.toBookieId());
         double observedMultiple1 = ((double) 
selectionCounts.get(addr4.toBookieId())

Reply via email to