You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by ni...@apache.org on 2022/04/06 12:45:04 UTC

[bookkeeper] branch branch-4.14 updated (1945fe0eb -> 84529a857)

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

nicoloboschi pushed a change to branch branch-4.14
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


    from 1945fe0eb [BRANCH-4.14] Replace Log4J with Reload4J
     new 79aae0476 catch onBookieRackChange exception (#3060)
     new 250108881 Fix region/rack aware placement police replace bookie bug (#2642)
     new 84529a857 fix region aware placement policy use disk weight not work (#2981)

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../RackawareEnsemblePlacementPolicyImpl.java      | 25 +++----
 .../client/RegionAwareEnsemblePlacementPolicy.java |  1 +
 .../TopologyAwareEnsemblePlacementPolicy.java      | 31 +++++---
 .../client/WeightedRandomSelectionImpl.java        |  2 -
 .../TestRegionAwareEnsemblePlacementPolicy.java    | 83 ++++++++++++++++++++++
 5 files changed, 119 insertions(+), 23 deletions(-)


[bookkeeper] 01/03: catch onBookieRackChange exception (#3060)

Posted by ni...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 79aae04766b5bca3a41bed6dc8e2e8ec4d0302bc
Author: Hang Chen <ch...@apache.org>
AuthorDate: Thu Mar 31 13:42:34 2022 +0800

    catch onBookieRackChange exception (#3060)
    
    ### Motivation
    When we update the bookie rack info, it will use all the bookie list to update rack topology. However If one bookie update failed and throw exception out, it will throw the exception out and the remains bookie info won't be updated into the rack topology, which will affect the ledger ensemble selection.
    
    ### Changes
    
    Catch the bookie topology update exception to ensure the remaining bookies' info can be updated into the rack topology.
    
    (cherry picked from commit f55ac1ed55336c3ba461c57a3428430309118344)
---
 .../TopologyAwareEnsemblePlacementPolicy.java      | 23 +++++++++++++---------
 1 file changed, 14 insertions(+), 9 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
index 438053f54..59bc893a9 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
@@ -47,6 +47,7 @@ import org.apache.bookkeeper.net.BookieSocketAddress;
 import org.apache.bookkeeper.net.DNSToSwitchMapping;
 import org.apache.bookkeeper.net.NetUtils;
 import org.apache.bookkeeper.net.NetworkTopology;
+import org.apache.bookkeeper.net.NetworkTopologyImpl;
 import org.apache.bookkeeper.net.Node;
 import org.apache.bookkeeper.net.NodeBase;
 import org.apache.bookkeeper.proto.BookieAddressResolver;
@@ -740,16 +741,20 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
     public void onBookieRackChange(List<BookieId> bookieAddressList) {
         rwLock.writeLock().lock();
         try {
-            for (BookieId bookieAddress : bookieAddressList) {
-                BookieNode node = knownBookies.get(bookieAddress);
-                if (node != null) {
-                    // refresh the rack info if its a known bookie
-                    BookieNode newNode = createBookieNode(bookieAddress);
-                    topology.remove(node);
-                    topology.add(newNode);
-                    knownBookies.put(bookieAddress, newNode);
+            bookieAddressList.forEach(bookieAddress -> {
+                try {
+                    BookieNode node = knownBookies.get(bookieAddress);
+                    if (node != null) {
+                        // refresh the rack info if its a known bookie
+                        BookieNode newNode = createBookieNode(bookieAddress);
+                        topology.remove(node);
+                        topology.add(newNode);
+                        knownBookies.put(bookieAddress, newNode);
+                    }
+                } catch (IllegalArgumentException | NetworkTopologyImpl.InvalidTopologyException e) {
+                    LOG.error("Failed to update bookie rack info: {} ", bookieAddress, e);
                 }
-            }
+            });
         } finally {
             rwLock.writeLock().unlock();
         }


[bookkeeper] 03/03: fix region aware placement policy use disk weight not work (#2981)

Posted by ni...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 84529a857a28ff6ee3c59437ce8f5b11a55ae56d
Author: Hang Chen <ch...@apache.org>
AuthorDate: Mon Jan 17 10:21:31 2022 +0800

    fix region aware placement policy use disk weight not work (#2981)
    
    ### Motivation
    When we meet the following conditions:
    1. configured region aware placement policy
    2. enable disk weight based placement
    3. fallback random selection when selecting ensemble bookies, such as:
         - not enough regions
         - rack number less than 2 in one region
    
    It will throw the following exception, and create ledger failed.
    ```
    12:15:36.459 [bookkeeper-ml-scheduler-OrderedScheduler-1-0] ERROR org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl - [public/default/persistent/test_v2] Encountered unexpected error when creating ledger
    java.lang.NullPointerException: null
            at org.apache.bookkeeper.client.WeightedRandomSelectionImpl.getNextRandom(WeightedRandomSelectionImpl.java:150) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandomInternal(RackawareEnsemblePlacementPolicyImpl.java:748) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.selectRandom(RackawareEnsemblePlacementPolicyImpl.java:698) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsembleInternal(RackawareEnsemblePlacementPolicyImpl.java:409) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicyImpl.newEnsemble(RackawareEnsemblePlacementPolicyImpl.java:372) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy.newEnsemble(RackawareEnsemblePlacementPolicy.java:159) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy.newEnsemble(RegionAwareEnsemblePlacementPolicy.java:303) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.BookieWatcherImpl.newEnsemble(BookieWatcherImpl.java:270) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.LedgerCreateOp.initiate(LedgerCreateOp.java:161) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.client.BookKeeper.asyncCreateLedger(BookKeeper.java:860) ~[io.streamnative-bookkeeper-server-4.14.3.1.jar:4.14.3.1]
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.asyncCreateLedger(ManagedLedgerImpl.java:3657) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.initializeBookKeeper(ManagedLedgerImpl.java:460) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.access$400(ManagedLedgerImpl.java:141) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$1.operationComplete(ManagedLedgerImpl.java:396) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl$1.operationComplete(ManagedLedgerImpl.java:328) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at org.apache.bookkeeper.mledger.impl.MetaStoreImpl.lambda$getManagedLedgerInfo$2(MetaStoreImpl.java:97) ~[io.streamnative-managed-ledger-2.8.1.30.jar:2.8.1.30]
            at java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714) [?:?]
            at java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:478) [?:?]
            at org.apache.bookkeeper.common.util.OrderedExecutor$TimedRunnable.run(OrderedExecutor.java:203) [io.streamnative-bookkeeper-common-4.14.3.1.jar:4.14.3.1]
            at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
            at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
            at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304) [?:?]
            at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) [?:?]
            at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) [?:?]
            at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) [io.netty-netty-common-4.1.72.Final.jar:4.1.72.Final]
            at java.lang.Thread.run(Thread.java:834) [?:?]
    ```
    
    The root cause of this case it that in `selectRandomInternal`, the `wRselection` haven't ever update any bookie map and the filed `randomMax` and `cummulativeMap` doesn't initialized.
    
    ### Modification
    1. update the `wRSelection`'s map on `selectRandomInternal` method whenever the `wRSelection` have ever set or not.
    
    (cherry picked from commit 002725ea5a25073de190bfe29f622d85fa264f0a)
---
 .../RackawareEnsemblePlacementPolicyImpl.java      | 25 ++++++++--------
 .../client/WeightedRandomSelectionImpl.java        |  2 --
 .../TestRegionAwareEnsemblePlacementPolicy.java    | 35 ++++++++++++++++++++++
 3 files changed, 48 insertions(+), 14 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 a4e3c8024..b561b9fe0 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
@@ -715,20 +715,21 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
                 throw new BKNotEnoughBookiesException();
             }
             if (wRSelection == null) {
-                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 {
-                        rackMap.put(n, new BookieInfo());
-                    }
-                }
                 wRSelection = new WeightedRandomSelectionImpl<BookieNode>(this.maxWeightMultiple);
-                wRSelection.updateMap(rackMap);
             }
+
+            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 {
+                    rackMap.put(n, new BookieInfo());
+                }
+            }
+            wRSelection.updateMap(rackMap);
         } else {
             Collections.shuffle(bookiesToSelectFrom);
         }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
index 40ad35e5c..8c2f1e2a3 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/WeightedRandomSelectionImpl.java
@@ -150,8 +150,6 @@ class WeightedRandomSelectionImpl<T> implements WeightedRandomSelection<T> {
             Double randomNum = randomMax * Math.random();
             // find the nearest key in the map corresponding to the randomNum
             Double key = cummulativeMap.floorKey(randomNum);
-            //LOG.info("Random max: {} CummulativeMap size: {} selected key: {}", randomMax, cummulativeMap.size(),
-            //    key);
             return cummulativeMap.get(key);
         } finally {
             rwLock.readLock().unlock();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
index 39649a66c..fc4e02cae 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
@@ -1572,4 +1572,39 @@ public class TestRegionAwareEnsemblePlacementPolicy extends TestCase {
             fail("Should not get not enough bookies exception even there is only one rack.");
         }
     }
+
+    public void testRegionsWithDiskWeight() throws Exception {
+        repp.uninitalize();
+        repp = new RegionAwareEnsemblePlacementPolicy();
+        conf.setProperty(REPP_ENABLE_VALIDATION, false);
+        conf.setDiskWeightBasedPlacementEnabled(true);
+        repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL,
+            NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER);
+        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);
+        BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181);
+        BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181);
+
+        // update dns mapping
+        StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1");
+        StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region2/r3");
+        StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region3/r11");
+        StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region4/r13");
+        StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/region5/r23");
+        // Update cluster
+        Set<BookieId> addrs = new HashSet<BookieId>();
+        addrs.add(addr1.toBookieId());
+        addrs.add(addr2.toBookieId());
+        addrs.add(addr3.toBookieId());
+        addrs.add(addr4.toBookieId());
+        addrs.add(addr5.toBookieId());
+
+        repp.onClusterChanged(addrs, new HashSet<BookieId>());
+
+        List<BookieId> ensemble = repp.newEnsemble(3, 3, 2, null,
+            new HashSet<>()).getResult();
+
+        assertEquals(3, ensemble.size());
+    }
 }


[bookkeeper] 02/03: Fix region/rack aware placement police replace bookie bug (#2642)

Posted by ni...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 25010888123a1b17c4aad41f04015ef35e1a69ac
Author: Hang Chen <ch...@apache.org>
AuthorDate: Thu Mar 31 21:39:47 2022 +0800

    Fix region/rack aware placement police replace bookie bug (#2642)
    
    (cherry picked from commit 4443c60739ad8f3ea2156c9bd7243e44a22709c9)
---
 .../client/RegionAwareEnsemblePlacementPolicy.java |  1 +
 .../TopologyAwareEnsemblePlacementPolicy.java      |  8 ++++
 .../TestRegionAwareEnsemblePlacementPolicy.java    | 48 ++++++++++++++++++++++
 3 files changed, 57 insertions(+)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
index 5c1b0a1a3..383260dc8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RegionAwareEnsemblePlacementPolicy.java
@@ -129,6 +129,7 @@ public class RegionAwareEnsemblePlacementPolicy extends RackawareEnsemblePlaceme
             BookieNode node = createBookieNode(addr);
             topology.add(node);
             knownBookies.put(addr, node);
+            historyBookies.put(addr, node);
             String region = getLocalRegion(node);
             if (null == perRegionPlacement.get(region)) {
                 perRegionPlacement.put(region, new RackawareEnsemblePlacementPolicy()
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
index 59bc893a9..7fdf3c5dd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TopologyAwareEnsemblePlacementPolicy.java
@@ -62,6 +62,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
     static final Logger LOG = LoggerFactory.getLogger(TopologyAwareEnsemblePlacementPolicy.class);
     public static final String REPP_DNS_RESOLVER_CLASS = "reppDnsResolverClass";
     protected final Map<BookieId, BookieNode> knownBookies = new HashMap<BookieId, BookieNode>();
+    protected final Map<BookieId, BookieNode> historyBookies = new HashMap<BookieId, BookieNode>();
     protected final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
     protected Map<BookieNode, WeightedObject> bookieInfoMap = new HashMap<BookieNode, WeightedObject>();
     // Initialize to empty set
@@ -717,6 +718,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
                 BookieNode node = createBookieNode(addr);
                 topology.add(node);
                 knownBookies.put(addr, node);
+                historyBookies.put(addr, node);
                 if (this.isWeighted) {
                     this.bookieInfoMap.putIfAbsent(node, new BookieInfo());
                 }
@@ -750,6 +752,7 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
                         topology.remove(node);
                         topology.add(newNode);
                         knownBookies.put(bookieAddress, newNode);
+                        historyBookies.put(bookieAddress, newNode);
                     }
                 } catch (IllegalArgumentException | NetworkTopologyImpl.InvalidTopologyException e) {
                     LOG.error("Failed to update bookie rack info: {} ", bookieAddress, e);
@@ -798,6 +801,11 @@ abstract class TopologyAwareEnsemblePlacementPolicy implements
         try {
             return NetUtils.resolveNetworkLocation(dnsResolver, bookieAddressResolver.resolve(addr));
         } catch (BookieAddressResolver.BookieIdNotResolvedException err) {
+            BookieNode historyBookie = historyBookies.get(addr);
+            if (null != historyBookie) {
+                return historyBookie.getNetworkLocation();
+            }
+
             LOG.error("Cannot resolve bookieId {} to a network address, resolving as {}", addr,
                       NetworkTopology.DEFAULT_REGION_AND_RACK, err);
             return NetworkTopology.DEFAULT_REGION_AND_RACK;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
index b96d3a488..39649a66c 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRegionAwareEnsemblePlacementPolicy.java
@@ -1421,6 +1421,54 @@ public class TestRegionAwareEnsemblePlacementPolicy extends TestCase {
         return numCoveredWriteQuorums;
     }
 
+    @Test
+    public void testRecoveryOnNodeFailure() throws Exception {
+        repp.uninitalize();
+        repp = new RegionAwareEnsemblePlacementPolicy();
+        repp.initialize(conf, Optional.empty(), timer, DISABLE_ALL,
+            NullStatsLogger.INSTANCE, BookieSocketAddress.LEGACY_BOOKIEID_RESOLVER);
+        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);
+        BookieSocketAddress addr4 = new BookieSocketAddress("127.0.0.5", 3181);
+        BookieSocketAddress addr5 = new BookieSocketAddress("127.0.0.6", 3181);
+        BookieSocketAddress addr6 = new BookieSocketAddress("127.0.0.7", 3181);
+
+        // Update dns mapping
+        StaticDNSResolver.addNodeToRack(addr1.getHostName(), "/region1/r1");
+        StaticDNSResolver.addNodeToRack(addr2.getHostName(), "/region1/r1");
+        StaticDNSResolver.addNodeToRack(addr3.getHostName(), "/region2/r2");
+        StaticDNSResolver.addNodeToRack(addr4.getHostName(), "/region2/r2");
+        StaticDNSResolver.addNodeToRack(addr5.getHostName(), "/region3/r3");
+        StaticDNSResolver.addNodeToRack(addr6.getHostName(), "/region3/r3");
+
+        // Update cluster
+        Set<BookieId> addrs = new HashSet<>();
+        addrs.add(addr1.toBookieId());
+        addrs.add(addr2.toBookieId());
+        addrs.add(addr3.toBookieId());
+        addrs.add(addr4.toBookieId());
+        addrs.add(addr5.toBookieId());
+        addrs.add(addr6.toBookieId());
+
+        repp.onClusterChanged(addrs, new HashSet<>());
+
+        Set<BookieId> bookiesLeftSet = new HashSet<>();
+        bookiesLeftSet.add(addr1.toBookieId());
+        repp.handleBookiesThatLeft(bookiesLeftSet);
+
+        List<BookieId> currentEnsemble = new ArrayList<>();
+        currentEnsemble.add(addr1.toBookieId());
+        currentEnsemble.add(addr3.toBookieId());
+        currentEnsemble.add(addr6.toBookieId());
+
+        EnsemblePlacementPolicy.PlacementResult<BookieId> placementResult = repp.replaceBookie(3,
+            3, 2, null,
+            currentEnsemble, addr1.toBookieId(), new HashSet<>());
+
+        assertEquals(placementResult.getResult(), addr2.toBookieId());
+    }
+
     @Test
     public void testNodeWithFailures() throws Exception {
         repp.uninitalize();