View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.zookeeper;
20  
21  import java.io.IOException;
22  import java.io.InputStream;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.Map.Entry;
26  import java.util.Properties;
27  
28  import com.google.common.annotations.VisibleForTesting;
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.HConstants;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  
35  /**
36   * Utility methods for reading, and building the ZooKeeper configuration.
37   */
38  @InterfaceAudience.Private
39  public final class ZKConfig {
40    private static final Log LOG = LogFactory.getLog(ZKConfig.class);
41  
42    private static final String VARIABLE_START = "${";
43    private static final int VARIABLE_START_LENGTH = VARIABLE_START.length();
44    private static final String VARIABLE_END = "}";
45    private static final int VARIABLE_END_LENGTH = VARIABLE_END.length();
46  
47    private ZKConfig() {
48    }
49  
50    /**
51     * Make a Properties object holding ZooKeeper config.
52     * Parses the corresponding config options from the HBase XML configs
53     * and generates the appropriate ZooKeeper properties.
54     * @param conf Configuration to read from.
55     * @return Properties holding mappings representing ZooKeeper config file.
56     */
57    public static Properties makeZKProps(Configuration conf) {
58      if (conf.getBoolean(HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG, false)) {
59        LOG.warn(
60            "Parsing ZooKeeper's " + HConstants.ZOOKEEPER_CONFIG_NAME +
61            " file for ZK properties " +
62            "has been deprecated. Please instead place all ZK related HBase " +
63            "configuration under the hbase-site.xml, using prefixes " +
64            "of the form '" + HConstants.ZK_CFG_PROPERTY_PREFIX + "', and " +
65            "set property '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG +
66            "' to false");
67        // First check if there is a zoo.cfg in the CLASSPATH. If so, simply read
68        // it and grab its configuration properties.
69        ClassLoader cl = ZKConfig.class.getClassLoader();
70        final InputStream inputStream =
71          cl.getResourceAsStream(HConstants.ZOOKEEPER_CONFIG_NAME);
72        if (inputStream != null) {
73          try {
74            return parseZooCfg(conf, inputStream);
75          } catch (IOException e) {
76            LOG.warn("Cannot read " + HConstants.ZOOKEEPER_CONFIG_NAME +
77                     ", loading from XML files", e);
78          }
79        }
80      } else {
81        if (LOG.isTraceEnabled()) {
82          LOG.trace("Skipped reading ZK properties file '" + HConstants.ZOOKEEPER_CONFIG_NAME +
83            "' since '" + HConstants.HBASE_CONFIG_READ_ZOOKEEPER_CONFIG + "' was not set to true");
84        }
85      }
86  
87      // Otherwise, use the configuration options from HBase's XML files.
88      Properties zkProperties = new Properties();
89  
90      // Directly map all of the hbase.zookeeper.property.KEY properties.
91      for (Entry<String, String> entry : new Configuration(conf)) { // copy for mt safety
92        String key = entry.getKey();
93        if (key.startsWith(HConstants.ZK_CFG_PROPERTY_PREFIX)) {
94          String zkKey = key.substring(HConstants.ZK_CFG_PROPERTY_PREFIX_LEN);
95          String value = entry.getValue();
96          // If the value has variables substitutions, need to do a get.
97          if (value.contains(VARIABLE_START)) {
98            value = conf.get(key);
99          }
100         zkProperties.put(zkKey, value);
101       }
102     }
103 
104     // If clientPort is not set, assign the default.
105     if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) {
106       zkProperties.put(HConstants.CLIENT_PORT_STR,
107           HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT);
108     }
109 
110     // Create the server.X properties.
111     int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
112     int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
113 
114     final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM,
115                                                  HConstants.LOCALHOST);
116     for (int i = 0; i < serverHosts.length; ++i) {
117       String serverHost = serverHosts[i];
118       String address = serverHost + ":" + peerPort + ":" + leaderPort;
119       String key = "server." + i;
120       zkProperties.put(key, address);
121     }
122 
123     return zkProperties;
124   }
125 
126   /**
127    * Parse ZooKeeper's zoo.cfg, injecting HBase Configuration variables in.
128    * This method is used for testing so we can pass our own InputStream.
129    * @param conf HBaseConfiguration to use for injecting variables.
130    * @param inputStream InputStream to read from.
131    * @return Properties parsed from config stream with variables substituted.
132    * @throws IOException if anything goes wrong parsing config
133    * @deprecated in 0.96 onwards. HBase will no longer rely on zoo.cfg
134    * availability.
135    */
136   @Deprecated
137   public static Properties parseZooCfg(Configuration conf,
138       InputStream inputStream) throws IOException {
139     Properties properties = new Properties();
140     try {
141       properties.load(inputStream);
142     } catch (IOException e) {
143       final String msg = "fail to read properties from "
144         + HConstants.ZOOKEEPER_CONFIG_NAME;
145       LOG.fatal(msg);
146       throw new IOException(msg, e);
147     }
148     for (Entry<Object, Object> entry : properties.entrySet()) {
149       String value = entry.getValue().toString().trim();
150       String key = entry.getKey().toString().trim();
151       StringBuilder newValue = new StringBuilder();
152       int varStart = value.indexOf(VARIABLE_START);
153       int varEnd = 0;
154       while (varStart != -1) {
155         varEnd = value.indexOf(VARIABLE_END, varStart);
156         if (varEnd == -1) {
157           String msg = "variable at " + varStart + " has no end marker";
158           LOG.fatal(msg);
159           throw new IOException(msg);
160         }
161         String variable = value.substring(varStart + VARIABLE_START_LENGTH, varEnd);
162 
163         String substituteValue = System.getProperty(variable);
164         if (substituteValue == null) {
165           substituteValue = conf.get(variable);
166         }
167         if (substituteValue == null) {
168           String msg = "variable " + variable + " not set in system property "
169                      + "or hbase configs";
170           LOG.fatal(msg);
171           throw new IOException(msg);
172         }
173 
174         newValue.append(substituteValue);
175 
176         varEnd += VARIABLE_END_LENGTH;
177         varStart = value.indexOf(VARIABLE_START, varEnd);
178       }
179       // Special case for 'hbase.cluster.distributed' property being 'true'
180       if (key.startsWith("server.")) {
181         boolean mode = conf.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED);
182         if (mode == HConstants.CLUSTER_IS_DISTRIBUTED && value.startsWith(HConstants.LOCALHOST)) {
183           String msg = "The server in zoo.cfg cannot be set to localhost " +
184               "in a fully-distributed setup because it won't be reachable. " +
185               "See \"Getting Started\" for more information.";
186           LOG.fatal(msg);
187           throw new IOException(msg);
188         }
189       }
190       newValue.append(value.substring(varEnd));
191       properties.setProperty(key, newValue.toString());
192     }
193     return properties;
194   }
195 
196   /**
197    * Return the ZK Quorum servers string given zk properties returned by
198    * makeZKProps
199    * @param properties
200    * @return Quorum servers String
201    */
202   public static String getZKQuorumServersString(Properties properties) {
203     String clientPort = null;
204     List<String> servers = new ArrayList<String>();
205 
206     // The clientPort option may come after the server.X hosts, so we need to
207     // grab everything and then create the final host:port comma separated list.
208     boolean anyValid = false;
209     for (Entry<Object,Object> property : properties.entrySet()) {
210       String key = property.getKey().toString().trim();
211       String value = property.getValue().toString().trim();
212       if (key.equals("clientPort")) {
213         clientPort = value;
214       }
215       else if (key.startsWith("server.")) {
216         String host = value.substring(0, value.indexOf(':'));
217         servers.add(host);
218         anyValid = true;
219       }
220     }
221 
222     if (!anyValid) {
223       LOG.error("no valid quorum servers found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
224       return null;
225     }
226 
227     if (clientPort == null) {
228       LOG.error("no clientPort found in " + HConstants.ZOOKEEPER_CONFIG_NAME);
229       return null;
230     }
231 
232     if (servers.isEmpty()) {
233       LOG.fatal("No servers were found in provided ZooKeeper configuration. " +
234           "HBase must have a ZooKeeper cluster configured for its " +
235           "operation. Ensure that you've configured '" +
236           HConstants.ZOOKEEPER_QUORUM + "' properly.");
237       return null;
238     }
239 
240     StringBuilder hostPortBuilder = new StringBuilder();
241     for (int i = 0; i < servers.size(); ++i) {
242       String host = servers.get(i);
243       if (i > 0) {
244         hostPortBuilder.append(',');
245       }
246       hostPortBuilder.append(host);
247       hostPortBuilder.append(':');
248       hostPortBuilder.append(clientPort);
249     }
250 
251     return hostPortBuilder.toString();
252   }
253 
254   /**
255    * Return the ZK Quorum servers string given the specified configuration
256    *
257    * @param conf
258    * @return Quorum servers String
259    */
260   private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) {
261     String defaultClientPort = Integer.toString(
262         conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT));
263 
264     // Build the ZK quorum server string with "server:clientport" list, separated by ','
265     final String[] serverHosts =
266         conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
267     return buildZKQuorumServerString(serverHosts, defaultClientPort);
268   }
269 
270   /**
271    * Return the ZK Quorum servers string given the specified configuration.
272    * @param conf
273    * @return Quorum servers
274    */
275   public static String getZKQuorumServersString(Configuration conf) {
276     // First try zoo.cfg; if not applicable, then try config XML.
277     Properties zkProperties = makeZKProps(conf);
278 
279     if (zkProperties != null) {
280       return getZKQuorumServersString(zkProperties);
281     }
282 
283     return getZKQuorumServersStringFromHbaseConfig(conf);
284   }
285 
286   /**
287    * Build the ZK quorum server string with "server:clientport" list, separated by ','
288    *
289    * @param serverHosts a list of servers for ZK quorum
290    * @param clientPort the default client port
291    * @return the string for a list of "server:port" separated by ","
292    */
293   public static String buildZKQuorumServerString(String[] serverHosts, String clientPort) {
294     StringBuilder quorumStringBuilder = new StringBuilder();
295     String serverHost;
296     for (int i = 0; i < serverHosts.length; ++i) {
297       if (serverHosts[i].contains(":")) {
298         serverHost = serverHosts[i]; // just use the port specified from the input
299       } else {
300         serverHost = serverHosts[i] + ":" + clientPort;
301       }
302       if (i > 0) {
303         quorumStringBuilder.append(',');
304       }
305       quorumStringBuilder.append(serverHost);
306     }
307     return quorumStringBuilder.toString();
308   }
309 
310   /**
311    * Verifies that the given key matches the expected format for a ZooKeeper cluster key.
312    * The Quorum for the ZK cluster can have one the following formats (see examples below):
313    *
314    * <ol>
315    *   <li>s1,s2,s3 (no client port in the list, the client port could be obtained from
316    *       clientPort)</li>
317    *   <li>s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
318    *       in this case, the clientPort would be ignored)</li>
319    *   <li>s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
320    *       the clientPort; otherwise, it would use the specified port)</li>
321    * </ol>
322    *
323    * @param key the cluster key to validate
324    * @throws IOException if the key could not be parsed
325    */
326   public static void validateClusterKey(String key) throws IOException {
327     transformClusterKey(key);
328   }
329 
330   /**
331    * Separate the given key into the three configurations it should contain:
332    * hbase.zookeeper.quorum, hbase.zookeeper.client.port
333    * and zookeeper.znode.parent
334    * @param key
335    * @return the three configuration in the described order
336    * @throws IOException
337    */
338   public static ZKClusterKey transformClusterKey(String key) throws IOException {
339     String[] parts = key.split(":");
340 
341     if (parts.length == 3) {
342       return new ZKClusterKey(parts [0], Integer.parseInt(parts [1]), parts [2]);
343     }
344 
345     if (parts.length > 3) {
346       // The quorum could contain client port in server:clientport format, try to transform more.
347       String zNodeParent = parts [parts.length - 1];
348       String clientPort = parts [parts.length - 2];
349 
350       // The first part length is the total length minus the lengths of other parts and minus 2 ":"
351       int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2;
352       String quorumStringInput = key.substring(0, endQuorumIndex);
353       String[] serverHosts = quorumStringInput.split(",");
354 
355       // The common case is that every server has its own client port specified - this means
356       // that (total parts - the ZNodeParent part - the ClientPort part) is equal to
357       // (the number of "," + 1) - "+ 1" because the last server has no ",".
358       if ((parts.length - 2) == (serverHosts.length + 1)) {
359         return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent);
360       }
361 
362       // For the uncommon case that some servers has no port specified, we need to build the
363       // server:clientport list using default client port for servers without specified port.
364       return new ZKClusterKey(
365           buildZKQuorumServerString(serverHosts, clientPort),
366           Integer.parseInt(clientPort),
367           zNodeParent);
368     }
369 
370     throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
371         HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
372         + HConstants.ZOOKEEPER_ZNODE_PARENT);
373   }
374 
375   /**
376    * Get the key to the ZK ensemble for this configuration without
377    * adding a name at the end
378    * @param conf Configuration to use to build the key
379    * @return ensemble key without a name
380    */
381   public static String getZooKeeperClusterKey(Configuration conf) {
382     return getZooKeeperClusterKey(conf, null);
383   }
384 
385   /**
386    * Get the key to the ZK ensemble for this configuration and append
387    * a name at the end
388    * @param conf Configuration to use to build the key
389    * @param name Name that should be appended at the end if not empty or null
390    * @return ensemble key with a name (if any)
391    */
392   public static String getZooKeeperClusterKey(Configuration conf, String name) {
393     String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM).replaceAll(
394         "[\\t\\n\\x0B\\f\\r]", "");
395     StringBuilder builder = new StringBuilder(ensemble);
396     builder.append(":");
397     builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
398     builder.append(":");
399     builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
400     if (name != null && !name.isEmpty()) {
401       builder.append(",");
402       builder.append(name);
403     }
404     return builder.toString();
405   }
406 
407   /**
408    * Standardize the ZK quorum string: make it a "server:clientport" list, separated by ','
409    * @param quorumStringInput a string contains a list of servers for ZK quorum
410    * @param clientPort the default client port
411    * @return the string for a list of "server:port" separated by ","
412    */
413   @VisibleForTesting
414   public static String standardizeZKQuorumServerString(String quorumStringInput,
415       String clientPort) {
416     String[] serverHosts = quorumStringInput.split(",");
417     return buildZKQuorumServerString(serverHosts, clientPort);
418   }
419 
420   // The Quorum for the ZK cluster can have one the following format (see examples below):
421   // (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort)
422   // (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
423   //      in this case, the clientPort would be ignored)
424   // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
425   //      the clientPort; otherwise, it would use the specified port)
426   @VisibleForTesting
427   public static class ZKClusterKey {
428     private String quorumString;
429     private int clientPort;
430     private String znodeParent;
431 
432     ZKClusterKey(String quorumString, int clientPort, String znodeParent) {
433       this.quorumString = quorumString;
434       this.clientPort = clientPort;
435       this.znodeParent = znodeParent;
436     }
437 
438     public String getQuorumString() {
439       return quorumString;
440     }
441 
442     public int getClientPort() {
443       return clientPort;
444     }
445 
446     public String getZnodeParent() {
447       return znodeParent;
448     }
449   }
450 }