From 358067eac8c8cf46f64fa9dc4722d44d4418cfeb Mon Sep 17 00:00:00 2001 From: Andrew Purtell Date: Tue, 15 Nov 2016 18:07:07 -0800 Subject: [PATCH] HBASE-17108 ZKConfig.getZKQuorumServersString does not return the correct client port number Port back the branch-1 version of ZKConfig, incorporating HBASE-15769 --- .../apache/hadoop/hbase/zookeeper/ZKConfig.java | 77 +++++++++++++++++----- 1 file changed, 61 insertions(+), 16 deletions(-) diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java index af6b98f..026db3b 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKConfig.java @@ -34,6 +34,11 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; /** * Utility methods for reading, and building the ZooKeeper configuration. + * + * The order and priority for reading the config are as follows: + * (1). zoo.cfg if ""hbase.config.read.zookeeper.config" is true + * (2). Property with "hbase.zookeeper.property." prefix from HBase XML + * (3). other zookeeper related properties in HBASE XML */ @InterfaceAudience.Private public final class ZKConfig { @@ -55,6 +60,24 @@ public final class ZKConfig { * @return Properties holding mappings representing ZooKeeper config file. */ public static Properties makeZKProps(Configuration conf) { + Properties zkProperties = makeZKPropsFromZooCfg(conf); + + if (zkProperties == null) { + // Otherwise, use the configuration options from HBase's XML files. + zkProperties = makeZKPropsFromHbaseConfig(conf); + } + return zkProperties; + } + + /** + * Parses the corresponding config options from the zoo.cfg file + * and make a Properties object holding the Zookeeper config. + * + * @param conf Configuration to read from. + * @return Properties holding mappings representing the ZooKeeper config file or null if + * the HBASE_CONFIG_READ_ZOOKEEPER_CONFIG is false or the file does not exist. + */ + private static Properties makeZKPropsFromZooCfg(Configuration conf) { if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, false)) { LOG.warn( "Parsing ZooKeeper's " + HConstants.ZOOKEEPER_CONFIG_NAME + @@ -84,20 +107,34 @@ public final class ZKConfig { } } - // Otherwise, use the configuration options from HBase's XML files. + return null; + } + + /** + * Make a Properties object holding ZooKeeper config. + * Parses the corresponding config options from the HBase XML configs + * and generates the appropriate ZooKeeper properties. + * + * @param conf Configuration to read from. + * @return Properties holding mappings representing ZooKeeper config file. + */ + private static Properties makeZKPropsFromHbaseConfig(Configuration conf) { Properties zkProperties = new Properties(); // Directly map all of the hbase.zookeeper.property.KEY properties. - for (Entry entry : new Configuration(conf)) { // copy for mt safety - String key = entry.getKey(); - if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) { - String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN); - String value = entry.getValue(); - // If the value has variables substitutions, need to do a get. - if (value.contains(VARIABLE_START)) { - value = conf.get(key); + // Synchronize on conf so no loading of configs while we iterate + synchronized (conf) { + for (Entry entry : conf) { + String key = entry.getKey(); + if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) { + String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN); + String value = entry.getValue(); + // If the value has variables substitutions, need to do a get. + if (value.contains(VARIABLE_START)) { + value = conf.get(key); + } + zkProperties.put(zkKey, value); } - zkProperties.put(zkKey, value); } } @@ -113,10 +150,17 @@ public final class ZKConfig { final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST); + String serverHost; + String address; + String key; for (int i = 0; i < serverHosts.length; ++i) { - String serverHost = serverHosts[i]; - String address = serverHost + ":" + peerPort + ":" + leaderPort; - String key = "server." + i; + if (serverHosts[i].contains(":")) { + serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':')); + } else { + serverHost = serverHosts[i]; + } + address = serverHost + ":" + peerPort + ":" + leaderPort; + key = "server." + i; zkProperties.put(key, address); } @@ -178,7 +222,8 @@ public final class ZKConfig { } // Special case for 'hbase.cluster.distributed' property being 'true' if (key.startsWith("server.")) { - boolean mode = conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED); + boolean mode = + conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED); if (mode == HConstants.CLUSTER_IS_DISTRIBUTED && value.startsWith(HConstants.LOCALHOST)) { String msg = "The server in zoo.cfg cannot be set to localhost " + "in a fully-distributed setup because it won't be reachable. " + @@ -199,7 +244,7 @@ public final class ZKConfig { * @param properties * @return Quorum servers String */ - public static String getZKQuorumServersString(Properties properties) { + static String getZKQuorumServersString(Properties properties) { String clientPort = null; List servers = new ArrayList(); @@ -274,7 +319,7 @@ public final class ZKConfig { */ public static String getZKQuorumServersString(Configuration conf) { // First try zoo.cfg; if not applicable, then try config XML. - Properties zkProperties = makeZKProps(conf); + Properties zkProperties = makeZKPropsFromZooCfg(conf); if (zkProperties != null) { return getZKQuorumServersString(zkProperties); -- 2.10.2