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 2019/02/27 07:50:49 UTC

[bookkeeper] branch master updated: Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method

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 756cae7  Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method
756cae7 is described below

commit 756cae7462919f03c7eea9dcf2985a1a6fdc64cd
Author: Charan Reddy Guttapalem <re...@gmail.com>
AuthorDate: Tue Feb 26 23:50:45 2019 -0800

    Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method
    
    
    
    Descriptions of the changes in this PR:
    
    - Use conf value in RackawareEnsemblePlacementPolicyImpl.initialize method,
    instead of uninitialized enforceMinNumRacksPerWriteQuorum variable
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1964 from reddycharan/fixexceptioncondition
---
 .../RackawareEnsemblePlacementPolicyImpl.java      |  2 +-
 ...ackawareEnsemblePlacementPolicyUsingScript.java | 45 ++++++++++++++++++++++
 2 files changed, 46 insertions(+), 1 deletion(-)

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 ef4341a..4cbe25b 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
@@ -386,7 +386,7 @@ public class RackawareEnsemblePlacementPolicyImpl extends TopologyAwareEnsembleP
                     ((RackChangeNotifier) dnsResolver).registerRackChangeListener(this);
                 }
             } catch (RuntimeException re) {
-                if (!enforceMinNumRacksPerWriteQuorum) {
+                if (!conf.getEnforceMinNumRacksPerWriteQuorum()) {
                     LOG.info("Failed to initialize DNS Resolver {}, used default subnet resolver : {}", dnsResolverName,
                             re, re.getMessage());
                     dnsResolver = new DefaultResolver(() -> this.getDefaultRack());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java
index 2aff5d8..fc3caaf 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestRackawareEnsemblePlacementPolicyUsingScript.java
@@ -358,6 +358,51 @@ public class TestRackawareEnsemblePlacementPolicyUsingScript {
         repp.onClusterChanged(addrs, new HashSet<BookieSocketAddress>());
     }
 
+    @Test
+    public void testNetworkTopologyScriptFileNameIsEmpty() throws Exception {
+        ignoreTestIfItIsWindowsOS();
+        repp.uninitalize();
+
+        ClientConfiguration newConf = new ClientConfiguration();
+        newConf.setProperty(REPP_DNS_RESOLVER_CLASS, ScriptBasedMapping.class.getName());
+        newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY, "");
+        newConf.setEnforceMinNumRacksPerWriteQuorum(false);
+        timer = new HashedWheelTimer(new ThreadFactoryBuilder().setNameFormat("TestTimer-%d").build(),
+                newConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS, newConf.getTimeoutTimerNumTicks());
+
+        repp = new RackawareEnsemblePlacementPolicy();
+        try {
+            repp.initialize(newConf, Optional.<DNSToSwitchMapping> empty(), timer, DISABLE_ALL,
+                    NullStatsLogger.INSTANCE);
+        } catch (RuntimeException re) {
+            fail("EnforceMinNumRacksPerWriteQuorum is not set, so repp.initialize should succeed even if"
+                    + " networkTopologyScriptFileName is empty");
+        }
+        repp.uninitalize();
+
+        newConf.setEnforceMinNumRacksPerWriteQuorum(true);
+        repp = new RackawareEnsemblePlacementPolicy();
+        try {
+            repp.initialize(newConf, Optional.<DNSToSwitchMapping> empty(), timer, DISABLE_ALL,
+                    NullStatsLogger.INSTANCE);
+            fail("EnforceMinNumRacksPerWriteQuorum is set, so repp.initialize should fail if"
+                    + " networkTopologyScriptFileName is empty");
+        } catch (RuntimeException re) {
+        }
+        repp.uninitalize();
+
+        newConf.setProperty(CommonConfigurationKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
+                "src/test/resources/networkmappingscript.sh");
+        try {
+            repp.initialize(newConf, Optional.<DNSToSwitchMapping> empty(), timer, DISABLE_ALL,
+                    NullStatsLogger.INSTANCE);
+        } catch (RuntimeException re) {
+            fail("EnforceMinNumRacksPerWriteQuorum is set and networkTopologyScriptFileName is not empty,"
+                    + " so it should succeed");
+        }
+        repp.uninitalize();
+    }
+
     private int getNumCoveredWriteQuorums(List<BookieSocketAddress> ensemble, int writeQuorumSize)
             throws Exception {
         int ensembleSize = ensemble.size();