You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/12/06 13:16:54 UTC

[bookkeeper] branch master updated: Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl

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

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


The following commit(s) were added to refs/heads/master by this push:
     new df857dd  Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl
df857dd is described below

commit df857dd678810e396ba44531e4c4546e67a0eb7a
Author: Charan Reddy Guttapalem <re...@gmail.com>
AuthorDate: Thu Dec 6 05:16:50 2018 -0800

    Fix selectFromNetworkLocation in RackawareEnsemblePlacementPolicyImpl
    
    
    Descriptions of the changes in this PR:
    
    Since beginning, selectFromNetworkLocation(excludeRacks, excludeBookies,..)
    method kind of ignores predicate/ensemble passed to that method
    https://github.com/apache/bookkeeper/blob/branch-4.7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L694.
    
    This was kind of ok, because so far effectively this method is called from only
    one place - https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/RackawareEnsemblePlacementPolicyImpl.java#L612,
    which passes TruePredicate.INSTANCE and EnsembleForReplacementWithNoConstraints.INSTANCE.
    
    But it is not ideal to ignore those parameters in selectFromNetworkLocation(excludeRacks, excludeBookies,..),
    from future usage perspective. So passing the received predicate and ensemble to the underlying calls.
    
    
    Reviewers: Sijie Guo <si...@apache.org>, Samuel Just <sj...@salesforce.com>
    
    This closes #1862 from reddycharan/fixrackaware
---
 .../RackawareEnsemblePlacementPolicyImpl.java      |   3 +-
 .../TestRackawareEnsemblePlacementPolicy.java      | 100 +++++++++++++++++++++
 2 files changed, 101 insertions(+), 2 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 155229a..578d49e 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
@@ -722,8 +722,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
         }
 
         try {
-            return selectRandomInternal(knownNodes, 1, fullExclusionBookiesList, TruePredicate.INSTANCE,
-                    EnsembleForReplacementWithNoConstraints.INSTANCE).get(0);
+            return selectRandomInternal(knownNodes, 1, fullExclusionBookiesList, predicate, ensemble).get(0);
         } catch (BKNotEnoughBookiesException e) {
             if (!fallbackToRandom) {
                 LOG.error(
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 d801044..e75e93f 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
@@ -41,6 +41,7 @@ import junit.framework.TestCase;
 
 import org.apache.bookkeeper.client.BKException.BKNotEnoughBookiesException;
 import org.apache.bookkeeper.client.BookieInfoReader.BookieInfo;
+import org.apache.bookkeeper.client.ITopologyAwareEnsemblePlacementPolicy.Ensemble;
 import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.BookieNode;
 import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.EnsembleForReplacementWithNoConstraints;
 import org.apache.bookkeeper.client.TopologyAwareEnsemblePlacementPolicy.TruePredicate;
@@ -1177,6 +1178,105 @@ public class TestRackawareEnsemblePlacementPolicy extends TestCase {
     }
 
     @Test
+    public void testSelectBookieByExcludingRacksAndBookies() throws Exception {
+        repp.uninitalize();
+
+        int minNumRacksPerWriteQuorum = 4;
+        ClientConfiguration clientConf = new ClientConfiguration(conf);
+        clientConf.setMinNumRacksPerWriteQuorum(minNumRacksPerWriteQuorum);
+        // set enforceMinNumRacksPerWriteQuorum
+        clientConf.setEnforceMinNumRacksPerWriteQuorum(true);
+        /*
+         * Durability is enforced
+         *
+         * When durability is being enforced; we must not violate the predicate
+         * even when selecting a random bookie; as durability guarantee is not
+         * best effort; correctness is implied by it
+         */
+        repp = new RackawareEnsemblePlacementPolicy(true);
+        repp.initialize(clientConf, Optional.<DNSToSwitchMapping> empty(), timer, DISABLE_ALL,
+                NullStatsLogger.INSTANCE);
+        repp.withDefaultRack(NetworkTopology.DEFAULT_REGION_AND_RACK);
+
+        int numOfRacks = 3;
+        int numOfBookiesPerRack = 5;
+        String[] rackLocationNames = new String[numOfRacks];
+        List<BookieSocketAddress> bookieSocketAddresses = new ArrayList<BookieSocketAddress>();
+        Map<BookieSocketAddress, String> bookieRackMap = new HashMap<BookieSocketAddress, String>();
+        BookieSocketAddress bookieAddress;
+
+        for (int i = 0; i < numOfRacks; i++) {
+            rackLocationNames[i] = "/default-region/r" + i;
+            for (int j = 0; j < numOfBookiesPerRack; j++) {
+                int index = i * numOfBookiesPerRack + j;
+                bookieAddress = new BookieSocketAddress("128.0.0." + index, 3181);
+                StaticDNSResolver.addNodeToRack(bookieAddress.getHostName(), rackLocationNames[i]);
+                bookieSocketAddresses.add(bookieAddress);
+                bookieRackMap.put(bookieAddress, rackLocationNames[i]);
+            }
+        }
+
+        repp.onClusterChanged(new HashSet<BookieSocketAddress>(bookieSocketAddresses),
+                new HashSet<BookieSocketAddress>());
+
+        Set<BookieSocketAddress> excludeBookiesOfRackR0 = new HashSet<BookieSocketAddress>();
+        for (int i = 0; i < numOfBookiesPerRack; i++) {
+            excludeBookiesOfRackR0.add(bookieSocketAddresses.get(i));
+        }
+
+        Set<Node> excludeBookieNodesOfRackR0 = repp.convertBookiesToNodes(excludeBookiesOfRackR0);
+
+        Set<String> excludeRackR1 = new HashSet<String>();
+        excludeRackR1.add(rackLocationNames[1]);
+
+        BookieNode nodeSelected;
+        nodeSelected = repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, TruePredicate.INSTANCE,
+                EnsembleForReplacementWithNoConstraints.INSTANCE, false);
+        assertEquals("BookieNode should be from Rack2", rackLocationNames[2], nodeSelected.getNetworkLocation());
+
+        try {
+            /*
+             * durability is enforced, so false predicate will reject all
+             * bookies.
+             */
+            repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, (candidate, chosenBookies) -> {
+                return false;
+            }, EnsembleForReplacementWithNoConstraints.INSTANCE, false);
+            fail("Should get not enough bookies exception since we are using false predicate");
+        } catch (BKNotEnoughBookiesException bnebe) {
+            // this is expected
+        }
+
+        try {
+            /*
+             * Using ensemble which rejects all the nodes.
+             */
+            repp.selectFromNetworkLocation(excludeRackR1, excludeBookieNodesOfRackR0, TruePredicate.INSTANCE,
+                    new Ensemble<BookieNode>() {
+
+                        @Override
+                        public boolean addNode(BookieNode node) {
+                            return false;
+                        }
+
+                        @Override
+                        public List<BookieSocketAddress> toList() {
+                            return null;
+                        }
+
+                        @Override
+                        public boolean validate() {
+                            return false;
+                        }
+
+                    }, false);
+            fail("Should get not enough bookies exception since ensemble rejects all the nodes");
+        } catch (BKNotEnoughBookiesException bnebe) {
+            // this is expected
+        }
+    }
+
+    @Test
     public void testNewEnsembleWithMultipleRacks() throws Exception {
         BookieSocketAddress addr1 = new BookieSocketAddress("127.0.0.1", 3181);
         BookieSocketAddress addr2 = new BookieSocketAddress("127.0.0.2", 3181);