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