001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.zookeeper; 019 020import java.io.IOException; 021import java.util.List; 022import java.util.Map.Entry; 023import java.util.Properties; 024import org.apache.hadoop.conf.Configuration; 025import org.apache.hadoop.hbase.HConstants; 026import org.apache.hadoop.util.StringUtils; 027import org.apache.yetus.audience.InterfaceAudience; 028import org.apache.zookeeper.client.ZKClientConfig; 029 030import org.apache.hbase.thirdparty.com.google.common.base.Splitter; 031 032/** 033 * Utility methods for reading, and building the ZooKeeper configuration. The order and priority for 034 * reading the config are as follows: 035 * <ol> 036 * <li>Property with "hbase.zookeeper.property." prefix from HBase XML.</li> 037 * <li>other zookeeper related properties in HBASE XML</li> 038 * </ol> 039 */ 040@InterfaceAudience.Private 041public final class ZKConfig { 042 043 private static final String VARIABLE_START = "${"; 044 045 private ZKConfig() { 046 } 047 048 /** 049 * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from 050 * the HBase XML configs and generates the appropriate ZooKeeper properties. 051 * @param conf Configuration to read from. 052 * @return Properties holding mappings representing ZooKeeper config file. 053 */ 054 public static Properties makeZKProps(Configuration conf) { 055 return makeZKPropsFromHbaseConfig(conf); 056 } 057 058 /** 059 * Directly map all the hbase.zookeeper.property.KEY properties. Synchronize on conf so no loading 060 * of configs while we iterate 061 */ 062 private static Properties extractZKPropsFromHBaseConfig(final Configuration conf) { 063 Properties zkProperties = new Properties(); 064 065 synchronized (conf) { 066 for (Entry<String, String> entry : conf) { 067 String key = entry.getKey(); 068 if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) { 069 String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN); 070 String value = entry.getValue(); 071 // If the value has variables substitutions, need to do a get. 072 if (value.contains(VARIABLE_START)) { 073 value = conf.get(key); 074 } 075 zkProperties.setProperty(zkKey, value); 076 } 077 } 078 } 079 080 return zkProperties; 081 } 082 083 /** 084 * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from 085 * the HBase XML configs and generates the appropriate ZooKeeper properties. 086 * @param conf Configuration to read from. 087 * @return Properties holding mappings representing ZooKeeper config file. 088 */ 089 private static Properties makeZKPropsFromHbaseConfig(Configuration conf) { 090 Properties zkProperties = extractZKPropsFromHBaseConfig(conf); 091 092 // If clientPort is not set, assign the default. 093 if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) { 094 zkProperties.put(HConstants.CLIENT_PORT_STR, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT); 095 } 096 097 // Create the server.X properties. 098 int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888); 099 int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888); 100 101 final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST); 102 String serverHost; 103 String address; 104 String key; 105 for (int i = 0; i < serverHosts.length; ++i) { 106 if (serverHosts[i].contains(":")) { 107 serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':')); 108 } else { 109 serverHost = serverHosts[i]; 110 } 111 address = serverHost + ":" + peerPort + ":" + leaderPort; 112 key = "server." + i; 113 zkProperties.put(key, address); 114 } 115 116 return zkProperties; 117 } 118 119 /** 120 * Return the ZK Quorum servers string given the specified configuration 121 * @return Quorum servers String 122 */ 123 private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) { 124 String defaultClientPort = Integer.toString( 125 conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT)); 126 127 // Build the ZK quorum server string with "server:clientport" list, separated by ',' 128 final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST); 129 return buildZKQuorumServerString(serverHosts, defaultClientPort); 130 } 131 132 /** 133 * Return the ZK Quorum servers string given the specified configuration. 134 * @return Quorum servers 135 */ 136 public static String getZKQuorumServersString(Configuration conf) { 137 return getZKQuorumServersStringFromHbaseConfig(conf); 138 } 139 140 /** 141 * Build the ZK quorum server string with "server:clientport" list, separated by ',' 142 * @param serverHosts a list of servers for ZK quorum 143 * @param clientPort the default client port 144 * @return the string for a list of "server:port" separated by "," 145 */ 146 public static String buildZKQuorumServerString(String[] serverHosts, String clientPort) { 147 StringBuilder quorumStringBuilder = new StringBuilder(); 148 String serverHost; 149 for (int i = 0; i < serverHosts.length; ++i) { 150 if (serverHosts[i].contains(":")) { 151 serverHost = serverHosts[i]; // just use the port specified from the input 152 } else { 153 serverHost = serverHosts[i] + ":" + clientPort; 154 } 155 if (i > 0) { 156 quorumStringBuilder.append(','); 157 } 158 quorumStringBuilder.append(serverHost); 159 } 160 return quorumStringBuilder.toString(); 161 } 162 163 /** 164 * Verifies that the given key matches the expected format for a ZooKeeper cluster key. The Quorum 165 * for the ZK cluster can have one the following formats (see examples below): 166 * <ol> 167 * <li>s1,s2,s3 (no client port in the list, the client port could be obtained from 168 * clientPort)</li> 169 * <li>s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server, in 170 * this case, the clientPort would be ignored)</li> 171 * <li>s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use the 172 * clientPort; otherwise, it would use the specified port)</li> 173 * </ol> 174 * @param key the cluster key to validate 175 * @throws IOException if the key could not be parsed 176 */ 177 public static void validateClusterKey(String key) throws IOException { 178 transformClusterKey(key); 179 } 180 181 /** 182 * Separate the given key into the three configurations it should contain: hbase.zookeeper.quorum, 183 * hbase.zookeeper.client.port and zookeeper.znode.parent 184 * @return the three configuration in the described order 185 */ 186 public static ZKClusterKey transformClusterKey(String key) throws IOException { 187 List<String> parts = Splitter.on(':').splitToList(key); 188 String[] partsArray = parts.toArray(new String[parts.size()]); 189 190 if (partsArray.length == 3) { 191 if (!partsArray[2].matches("/.*[^/]")) { 192 throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" 193 + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":" 194 + HConstants.ZOOKEEPER_ZNODE_PARENT); 195 } 196 return new ZKClusterKey(partsArray[0], Integer.parseInt(partsArray[1]), partsArray[2]); 197 } 198 199 if (partsArray.length > 3) { 200 // The quorum could contain client port in server:clientport format, try to transform more. 201 String zNodeParent = partsArray[partsArray.length - 1]; 202 if (!zNodeParent.matches("/.*[^/]")) { 203 throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" 204 + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":" 205 + HConstants.ZOOKEEPER_ZNODE_PARENT); 206 } 207 208 String clientPort = partsArray[partsArray.length - 2]; 209 210 // The first part length is the total length minus the lengths of other parts and minus 2 ":" 211 int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2; 212 String quorumStringInput = key.substring(0, endQuorumIndex); 213 String[] serverHosts = quorumStringInput.split(","); 214 215 // The common case is that every server has its own client port specified - this means 216 // that (total parts - the ZNodeParent part - the ClientPort part) is equal to 217 // (the number of "," + 1) - "+ 1" because the last server has no ",". 218 if ((partsArray.length - 2) == (serverHosts.length + 1)) { 219 return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent); 220 } 221 222 // For the uncommon case that some servers has no port specified, we need to build the 223 // server:clientport list using default client port for servers without specified port. 224 return new ZKClusterKey(buildZKQuorumServerString(serverHosts, clientPort), 225 Integer.parseInt(clientPort), zNodeParent); 226 } 227 228 throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" 229 + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":" 230 + HConstants.ZOOKEEPER_ZNODE_PARENT); 231 } 232 233 /** 234 * Get the key to the ZK ensemble for this configuration without adding a name at the end 235 * @param conf Configuration to use to build the key 236 * @return ensemble key without a name 237 */ 238 public static String getZooKeeperClusterKey(Configuration conf) { 239 return getZooKeeperClusterKey(conf, null); 240 } 241 242 /** 243 * Get the key to the ZK ensemble for this configuration and append a name at the end 244 * @param conf Configuration to use to build the key 245 * @param name Name that should be appended at the end if not empty or null 246 * @return ensemble key with a name (if any) 247 */ 248 public static String getZooKeeperClusterKey(Configuration conf, String name) { 249 String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM).replaceAll("[\\t\\n\\x0B\\f\\r]", ""); 250 StringBuilder builder = new StringBuilder(ensemble); 251 builder.append(":"); 252 builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT)); 253 builder.append(":"); 254 builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT)); 255 if (name != null && !name.isEmpty()) { 256 builder.append(","); 257 builder.append(name); 258 } 259 return builder.toString(); 260 } 261 262 /** 263 * Standardize the ZK quorum string: make it a "server:clientport" list, separated by ',' 264 * @param quorumStringInput a string contains a list of servers for ZK quorum 265 * @param clientPort the default client port 266 * @return the string for a list of "server:port" separated by "," 267 */ 268 public static String standardizeZKQuorumServerString(String quorumStringInput, 269 String clientPort) { 270 String[] serverHosts = quorumStringInput.split(","); 271 return buildZKQuorumServerString(serverHosts, clientPort); 272 } 273 274 // The Quorum for the ZK cluster can have one the following format (see examples below): 275 // (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort) 276 // (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server, 277 // in this case, the clientPort would be ignored) 278 // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use 279 // the clientPort; otherwise, it would use the specified port) 280 public static class ZKClusterKey { 281 private String quorumString; 282 private int clientPort; 283 private String znodeParent; 284 285 ZKClusterKey(String quorumString, int clientPort, String znodeParent) { 286 this.quorumString = quorumString; 287 this.clientPort = clientPort; 288 this.znodeParent = znodeParent; 289 } 290 291 public String getQuorumString() { 292 return quorumString; 293 } 294 295 public int getClientPort() { 296 return clientPort; 297 } 298 299 public String getZnodeParent() { 300 return znodeParent; 301 } 302 } 303 304 public static ZKClientConfig getZKClientConfig(Configuration conf) { 305 Properties zkProperties = extractZKPropsFromHBaseConfig(conf); 306 ZKClientConfig zkClientConfig = new ZKClientConfig(); 307 zkProperties.forEach((k, v) -> zkClientConfig.setProperty(k.toString(), v.toString())); 308 return zkClientConfig; 309 } 310 311 /** 312 * Get the client ZK Quorum servers string 313 * @param conf the configuration to read 314 * @return Client quorum servers, or null if not specified 315 */ 316 public static String getClientZKQuorumServersString(Configuration conf) { 317 String clientQuromServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM); 318 if (clientQuromServers == null) { 319 return null; 320 } 321 int defaultClientPort = 322 conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT); 323 String clientZkClientPort = 324 Integer.toString(conf.getInt(HConstants.CLIENT_ZOOKEEPER_CLIENT_PORT, defaultClientPort)); 325 // Build the ZK quorum server string with "server:clientport" list, separated by ',' 326 final String[] serverHosts = StringUtils.getStrings(clientQuromServers); 327 return buildZKQuorumServerString(serverHosts, clientZkClientPort); 328 } 329}