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;
028
029import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
030
031/**
032 * Utility methods for reading, and building the ZooKeeper configuration. The order and priority for
033 * reading the config are as follows: (1). Property with "hbase.zookeeper.property." prefix from
034 * HBase XML (2). other zookeeper related properties in HBASE XML
035 */
036@InterfaceAudience.Private
037public final class ZKConfig {
038
039  private static final String VARIABLE_START = "${";
040
041  private ZKConfig() {
042  }
043
044  /**
045   * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from
046   * the HBase XML configs and generates the appropriate ZooKeeper properties.
047   * @param conf Configuration to read from.
048   * @return Properties holding mappings representing ZooKeeper config file.
049   */
050  public static Properties makeZKProps(Configuration conf) {
051    return makeZKPropsFromHbaseConfig(conf);
052  }
053
054  /**
055   * Make a Properties object holding ZooKeeper config. Parses the corresponding config options from
056   * the HBase XML configs and generates the appropriate ZooKeeper properties.
057   * @param conf Configuration to read from.
058   * @return Properties holding mappings representing ZooKeeper config file.
059   */
060  private static Properties makeZKPropsFromHbaseConfig(Configuration conf) {
061    Properties zkProperties = new Properties();
062
063    // Directly map all of the hbase.zookeeper.property.KEY properties.
064    // Synchronize on conf so no loading of configs while we iterate
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    // If clientPort is not set, assign the default.
081    if (zkProperties.getProperty(HConstants.CLIENT_PORT_STR) == null) {
082      zkProperties.put(HConstants.CLIENT_PORT_STR, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT);
083    }
084
085    // Create the server.X properties.
086    int peerPort = conf.getInt("hbase.zookeeper.peerport", 2888);
087    int leaderPort = conf.getInt("hbase.zookeeper.leaderport", 3888);
088
089    final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
090    String serverHost;
091    String address;
092    String key;
093    for (int i = 0; i < serverHosts.length; ++i) {
094      if (serverHosts[i].contains(":")) {
095        serverHost = serverHosts[i].substring(0, serverHosts[i].indexOf(':'));
096      } else {
097        serverHost = serverHosts[i];
098      }
099      address = serverHost + ":" + peerPort + ":" + leaderPort;
100      key = "server." + i;
101      zkProperties.put(key, address);
102    }
103
104    return zkProperties;
105  }
106
107  /**
108   * Return the ZK Quorum servers string given the specified configuration n * @return Quorum
109   * servers String
110   */
111  private static String getZKQuorumServersStringFromHbaseConfig(Configuration conf) {
112    String defaultClientPort = Integer.toString(
113      conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT));
114
115    // Build the ZK quorum server string with "server:clientport" list, separated by ','
116    final String[] serverHosts = conf.getStrings(HConstants.ZOOKEEPER_QUORUM, HConstants.LOCALHOST);
117    return buildZKQuorumServerString(serverHosts, defaultClientPort);
118  }
119
120  /**
121   * Return the ZK Quorum servers string given the specified configuration.
122   * @return Quorum servers
123   */
124  public static String getZKQuorumServersString(Configuration conf) {
125    return getZKQuorumServersStringFromHbaseConfig(conf);
126  }
127
128  /**
129   * Build the ZK quorum server string with "server:clientport" list, separated by ','
130   * @param serverHosts a list of servers for ZK quorum
131   * @param clientPort  the default client port
132   * @return the string for a list of "server:port" separated by ","
133   */
134  public static String buildZKQuorumServerString(String[] serverHosts, String clientPort) {
135    StringBuilder quorumStringBuilder = new StringBuilder();
136    String serverHost;
137    for (int i = 0; i < serverHosts.length; ++i) {
138      if (serverHosts[i].contains(":")) {
139        serverHost = serverHosts[i]; // just use the port specified from the input
140      } else {
141        serverHost = serverHosts[i] + ":" + clientPort;
142      }
143      if (i > 0) {
144        quorumStringBuilder.append(',');
145      }
146      quorumStringBuilder.append(serverHost);
147    }
148    return quorumStringBuilder.toString();
149  }
150
151  /**
152   * Verifies that the given key matches the expected format for a ZooKeeper cluster key. The Quorum
153   * for the ZK cluster can have one the following formats (see examples below):
154   * <ol>
155   * <li>s1,s2,s3 (no client port in the list, the client port could be obtained from
156   * clientPort)</li>
157   * <li>s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server, in
158   * this case, the clientPort would be ignored)</li>
159   * <li>s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use the
160   * clientPort; otherwise, it would use the specified port)</li>
161   * </ol>
162   * @param key the cluster key to validate
163   * @throws IOException if the key could not be parsed
164   */
165  public static void validateClusterKey(String key) throws IOException {
166    transformClusterKey(key);
167  }
168
169  /**
170   * Separate the given key into the three configurations it should contain: hbase.zookeeper.quorum,
171   * hbase.zookeeper.client.port and zookeeper.znode.parent n * @return the three configuration in
172   * the described order n
173   */
174  public static ZKClusterKey transformClusterKey(String key) throws IOException {
175    List<String> parts = Splitter.on(':').splitToList(key);
176    String[] partsArray = parts.toArray(new String[parts.size()]);
177
178    if (partsArray.length == 3) {
179      if (!partsArray[2].matches("/.*[^/]")) {
180        throw new IOException("Cluster key passed " + key + " is invalid, the format should be:"
181          + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
182          + HConstants.ZOOKEEPER_ZNODE_PARENT);
183      }
184      return new ZKClusterKey(partsArray[0], Integer.parseInt(partsArray[1]), partsArray[2]);
185    }
186
187    if (partsArray.length > 3) {
188      // The quorum could contain client port in server:clientport format, try to transform more.
189      String zNodeParent = partsArray[partsArray.length - 1];
190      if (!zNodeParent.matches("/.*[^/]")) {
191        throw new IOException("Cluster key passed " + key + " is invalid, the format should be:"
192          + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
193          + HConstants.ZOOKEEPER_ZNODE_PARENT);
194      }
195
196      String clientPort = partsArray[partsArray.length - 2];
197
198      // The first part length is the total length minus the lengths of other parts and minus 2 ":"
199      int endQuorumIndex = key.length() - zNodeParent.length() - clientPort.length() - 2;
200      String quorumStringInput = key.substring(0, endQuorumIndex);
201      String[] serverHosts = quorumStringInput.split(",");
202
203      // The common case is that every server has its own client port specified - this means
204      // that (total parts - the ZNodeParent part - the ClientPort part) is equal to
205      // (the number of "," + 1) - "+ 1" because the last server has no ",".
206      if ((partsArray.length - 2) == (serverHosts.length + 1)) {
207        return new ZKClusterKey(quorumStringInput, Integer.parseInt(clientPort), zNodeParent);
208      }
209
210      // For the uncommon case that some servers has no port specified, we need to build the
211      // server:clientport list using default client port for servers without specified port.
212      return new ZKClusterKey(buildZKQuorumServerString(serverHosts, clientPort),
213        Integer.parseInt(clientPort), zNodeParent);
214    }
215
216    throw new IOException("Cluster key passed " + key + " is invalid, the format should be:"
217      + HConstants.ZOOKEEPER_QUORUM + ":" + HConstants.ZOOKEEPER_CLIENT_PORT + ":"
218      + HConstants.ZOOKEEPER_ZNODE_PARENT);
219  }
220
221  /**
222   * Get the key to the ZK ensemble for this configuration without adding a name at the end
223   * @param conf Configuration to use to build the key
224   * @return ensemble key without a name
225   */
226  public static String getZooKeeperClusterKey(Configuration conf) {
227    return getZooKeeperClusterKey(conf, null);
228  }
229
230  /**
231   * Get the key to the ZK ensemble for this configuration and append a name at the end
232   * @param conf Configuration to use to build the key
233   * @param name Name that should be appended at the end if not empty or null
234   * @return ensemble key with a name (if any)
235   */
236  public static String getZooKeeperClusterKey(Configuration conf, String name) {
237    String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM).replaceAll("[\\t\\n\\x0B\\f\\r]", "");
238    StringBuilder builder = new StringBuilder(ensemble);
239    builder.append(":");
240    builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
241    builder.append(":");
242    builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
243    if (name != null && !name.isEmpty()) {
244      builder.append(",");
245      builder.append(name);
246    }
247    return builder.toString();
248  }
249
250  /**
251   * Standardize the ZK quorum string: make it a "server:clientport" list, separated by ','
252   * @param quorumStringInput a string contains a list of servers for ZK quorum
253   * @param clientPort        the default client port
254   * @return the string for a list of "server:port" separated by ","
255   */
256  public static String standardizeZKQuorumServerString(String quorumStringInput,
257    String clientPort) {
258    String[] serverHosts = quorumStringInput.split(",");
259    return buildZKQuorumServerString(serverHosts, clientPort);
260  }
261
262  // The Quorum for the ZK cluster can have one the following format (see examples below):
263  // (1). s1,s2,s3 (no client port in the list, the client port could be obtained from clientPort)
264  // (2). s1:p1,s2:p2,s3:p3 (with client port, which could be same or different for each server,
265  // in this case, the clientPort would be ignored)
266  // (3). s1:p1,s2,s3:p3 (mix of (1) and (2) - if port is not specified in a server, it would use
267  // the clientPort; otherwise, it would use the specified port)
268  public static class ZKClusterKey {
269    private String quorumString;
270    private int clientPort;
271    private String znodeParent;
272
273    ZKClusterKey(String quorumString, int clientPort, String znodeParent) {
274      this.quorumString = quorumString;
275      this.clientPort = clientPort;
276      this.znodeParent = znodeParent;
277    }
278
279    public String getQuorumString() {
280      return quorumString;
281    }
282
283    public int getClientPort() {
284      return clientPort;
285    }
286
287    public String getZnodeParent() {
288      return znodeParent;
289    }
290  }
291
292  /**
293   * Get the client ZK Quorum servers string
294   * @param conf the configuration to read
295   * @return Client quorum servers, or null if not specified
296   */
297  public static String getClientZKQuorumServersString(Configuration conf) {
298    String clientQuromServers = conf.get(HConstants.CLIENT_ZOOKEEPER_QUORUM);
299    if (clientQuromServers == null) {
300      return null;
301    }
302    int defaultClientPort =
303      conf.getInt(HConstants.ZOOKEEPER_CLIENT_PORT, HConstants.DEFAULT_ZOOKEEPER_CLIENT_PORT);
304    String clientZkClientPort =
305      Integer.toString(conf.getInt(HConstants.CLIENT_ZOOKEEPER_CLIENT_PORT, defaultClientPort));
306    // Build the ZK quorum server string with "server:clientport" list, separated by ','
307    final String[] serverHosts = StringUtils.getStrings(clientQuromServers);
308    return buildZKQuorumServerString(serverHosts, clientZkClientPort);
309  }
310}