View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.zookeeper;
21  
22  import java.io.BufferedReader;
23  import java.io.IOException;
24  import java.io.InputStreamReader;
25  import java.io.PrintWriter;
26  import java.net.InetSocketAddress;
27  import java.net.Socket;
28  import java.util.ArrayList;
29  import java.util.List;
30  import java.util.Properties;
31  
32  import org.apache.commons.lang.StringUtils;
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.executor.RegionTransitionData;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.zookeeper.AsyncCallback;
40  import org.apache.zookeeper.CreateMode;
41  import org.apache.zookeeper.KeeperException;
42  import org.apache.zookeeper.Watcher;
43  import org.apache.zookeeper.KeeperException.NoNodeException;
44  import org.apache.zookeeper.ZooDefs.Ids;
45  import org.apache.zookeeper.data.ACL;
46  import org.apache.zookeeper.data.Stat;
47  
48  /**
49   * Internal HBase utility class for ZooKeeper.
50   *
51   * <p>Contains only static methods and constants.
52   *
53   * <p>Methods all throw {@link KeeperException} if there is an unexpected
54   * zookeeper exception, so callers of these methods must handle appropriately.
55   * If ZK is required for the operation, the server will need to be aborted.
56   */
57  public class ZKUtil {
58    private static final Log LOG = LogFactory.getLog(ZKUtil.class);
59  
60    // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
61    private static final char ZNODE_PATH_SEPARATOR = '/';
62    private static int zkDumpConnectionTimeOut;
63  
64    /**
65     * Creates a new connection to ZooKeeper, pulling settings and ensemble config
66     * from the specified configuration object using methods from {@link ZKConfig}.
67     *
68     * Sets the connection status monitoring watcher to the specified watcher.
69     *
70     * @param conf configuration to pull ensemble and other settings from
71     * @param watcher watcher to monitor connection changes
72     * @return connection to zookeeper
73     * @throws IOException if unable to connect to zk or config problem
74     */
75    public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
76    throws IOException {
77      Properties properties = ZKConfig.makeZKProps(conf);
78      String ensemble = ZKConfig.getZKQuorumServersString(properties);
79      return connect(conf, ensemble, watcher);
80    }
81  
82    public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
83        Watcher watcher)
84    throws IOException {
85      return connect(conf, ensemble, watcher, "");
86    }
87  
88    public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
89        Watcher watcher, final String descriptor)
90    throws IOException {
91      if(ensemble == null) {
92        throw new IOException("Unable to determine ZooKeeper ensemble");
93      }
94      int timeout = conf.getInt("zookeeper.session.timeout", 180 * 1000);
95      LOG.debug(descriptor + " opening connection to ZooKeeper with ensemble (" +
96          ensemble + ")");
97      int retry = conf.getInt("zookeeper.recovery.retry", 3);
98      int retryIntervalMillis =
99        conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
100     zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
101         1000);
102     return new RecoverableZooKeeper(ensemble, timeout, watcher,
103         retry, retryIntervalMillis);
104   }
105 
106   //
107   // Helper methods
108   //
109 
110   /**
111    * Join the prefix znode name with the suffix znode name to generate a proper
112    * full znode name.
113    *
114    * Assumes prefix does not end with slash and suffix does not begin with it.
115    *
116    * @param prefix beginning of znode name
117    * @param suffix ending of znode name
118    * @return result of properly joining prefix with suffix
119    */
120   public static String joinZNode(String prefix, String suffix) {
121     return prefix + ZNODE_PATH_SEPARATOR + suffix;
122   }
123 
124   /**
125    * Returns the full path of the immediate parent of the specified node.
126    * @param node path to get parent of
127    * @return parent of path, null if passed the root node or an invalid node
128    */
129   public static String getParent(String node) {
130     int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR);
131     return idx <= 0 ? null : node.substring(0, idx);
132   }
133 
134   /**
135    * Get the name of the current node from the specified fully-qualified path.
136    * @param path fully-qualified path
137    * @return name of the current node
138    */
139   public static String getNodeName(String path) {
140     return path.substring(path.lastIndexOf("/")+1);
141   }
142 
143   /**
144    * Get the key to the ZK ensemble for this configuration without
145    * adding a name at the end
146    * @param conf Configuration to use to build the key
147    * @return ensemble key without a name
148    */
149   public static String getZooKeeperClusterKey(Configuration conf) {
150     return getZooKeeperClusterKey(conf, null);
151   }
152 
153   /**
154    * Get the key to the ZK ensemble for this configuration and append
155    * a name at the end
156    * @param conf Configuration to use to build the key
157    * @param name Name that should be appended at the end if not empty or null
158    * @return ensemble key with a name (if any)
159    */
160   public static String getZooKeeperClusterKey(Configuration conf, String name) {
161     String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM.replaceAll(
162         "[\\t\\n\\x0B\\f\\r]", ""));
163     StringBuilder builder = new StringBuilder(ensemble);
164     builder.append(":");
165     builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
166     builder.append(":");
167     builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
168     if (name != null && !name.isEmpty()) {
169       builder.append(",");
170       builder.append(name);
171     }
172     return builder.toString();
173   }
174 
175   /**
176    * Apply the settings in the given key to the given configuration, this is
177    * used to communicate with distant clusters
178    * @param conf configuration object to configure
179    * @param key string that contains the 3 required configuratins
180    * @throws IOException
181    */
182   public static void applyClusterKeyToConf(Configuration conf, String key)
183       throws IOException{
184     String[] parts = transformClusterKey(key);
185     conf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
186     conf.set("hbase.zookeeper.property.clientPort", parts[1]);
187     conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[2]);
188   }
189 
190   /**
191    * Separate the given key into the three configurations it should contain:
192    * hbase.zookeeper.quorum, hbase.zookeeper.client.port
193    * and zookeeper.znode.parent
194    * @param key
195    * @return the three configuration in the described order
196    * @throws IOException
197    */
198   public static String[] transformClusterKey(String key) throws IOException {
199     String[] parts = key.split(":");
200     if (parts.length != 3) {
201       throw new IOException("Cluster key invalid, the format should be:" +
202           HConstants.ZOOKEEPER_QUORUM + ":hbase.zookeeper.client.port:"
203           + HConstants.ZOOKEEPER_ZNODE_PARENT);
204     }
205     return parts;
206   }
207 
208   //
209   // Existence checks and watches
210   //
211 
212   /**
213    * Watch the specified znode for delete/create/change events.  The watcher is
214    * set whether or not the node exists.  If the node already exists, the method
215    * returns true.  If the node does not exist, the method returns false.
216    *
217    * @param zkw zk reference
218    * @param znode path of node to watch
219    * @return true if znode exists, false if does not exist or error
220    * @throws KeeperException if unexpected zookeeper exception
221    */
222   public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode)
223   throws KeeperException {
224     try {
225       Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
226       boolean exists = s != null ? true : false;
227       if (exists) {
228         LOG.debug(zkw.prefix("Set watcher on existing znode " + znode));
229       } else {
230         LOG.debug(zkw.prefix(znode+" does not exist. Watcher is set."));
231       }
232       return exists;
233     } catch (KeeperException e) {
234       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
235       zkw.keeperException(e);
236       return false;
237     } catch (InterruptedException e) {
238       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
239       zkw.interruptedException(e);
240       return false;
241     }
242   }
243 
244   /**
245    * Check if the specified node exists.  Sets no watches.
246    *
247    * Returns true if node exists, false if not.  Returns an exception if there
248    * is an unexpected zookeeper exception.
249    *
250    * @param zkw zk reference
251    * @param znode path of node to watch
252    * @return version of the node if it exists, -1 if does not exist
253    * @throws KeeperException if unexpected zookeeper exception
254    */
255   public static int checkExists(ZooKeeperWatcher zkw, String znode)
256   throws KeeperException {
257     try {
258       Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
259       return s != null ? s.getVersion() : -1;
260     } catch (KeeperException e) {
261       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
262       zkw.keeperException(e);
263       return -1;
264     } catch (InterruptedException e) {
265       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
266       zkw.interruptedException(e);
267       return -1;
268     }
269   }
270 
271   //
272   // Znode listings
273   //
274 
275   /**
276    * Lists the children znodes of the specified znode.  Also sets a watch on
277    * the specified znode which will capture a NodeDeleted event on the specified
278    * znode as well as NodeChildrenChanged if any children of the specified znode
279    * are created or deleted.
280    *
281    * Returns null if the specified node does not exist.  Otherwise returns a
282    * list of children of the specified node.  If the node exists but it has no
283    * children, an empty list will be returned.
284    *
285    * @param zkw zk reference
286    * @param znode path of node to list and watch children of
287    * @return list of children of the specified node, an empty list if the node
288    *          exists but has no children, and null if the node does not exist
289    * @throws KeeperException if unexpected zookeeper exception
290    */
291   public static List<String> listChildrenAndWatchForNewChildren(
292       ZooKeeperWatcher zkw, String znode)
293   throws KeeperException {
294     try {
295       List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
296       return children;
297     } catch(KeeperException.NoNodeException ke) {
298       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
299           "because node does not exist (not an error)"));
300       return null;
301     } catch (KeeperException e) {
302       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
303       zkw.keeperException(e);
304       return null;
305     } catch (InterruptedException e) {
306       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
307       zkw.interruptedException(e);
308       return null;
309     }
310   }
311 
312   /**
313    * List all the children of the specified znode, setting a watch for children
314    * changes and also setting a watch on every individual child in order to get
315    * the NodeCreated and NodeDeleted events.
316    * @param zkw zookeeper reference
317    * @param znode node to get children of and watch
318    * @return list of znode names, null if the node doesn't exist
319    * @throws KeeperException
320    */
321   public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
322       String znode) throws KeeperException {
323     List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
324     if (children == null) {
325       return null;
326     }
327     for (String child : children) {
328       watchAndCheckExists(zkw, joinZNode(znode, child));
329     }
330     return children;
331   }
332 
333   /**
334    * Lists the children of the specified znode without setting any watches.
335    *
336    * Used to list the currently online regionservers and their addresses.
337    *
338    * Sets no watches at all, this method is best effort.
339    *
340    * Returns an empty list if the node has no children.  Returns null if the
341    * parent node itself does not exist.
342    *
343    * @param zkw zookeeper reference
344    * @param znode node to get children of as addresses
345    * @return list of data of children of specified znode, empty if no children,
346    *         null if parent does not exist
347    * @throws KeeperException if unexpected zookeeper exception
348    */
349   public static List<String> listChildrenNoWatch(
350       ZooKeeperWatcher zkw, String znode)
351   throws KeeperException {
352     List<String> children = null;
353     try {
354       // List the children without watching
355       children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
356     } catch(KeeperException.NoNodeException nne) {
357       return null;
358     } catch(InterruptedException ie) {
359       zkw.interruptedException(ie);
360     }
361     return children;
362   }
363 
364   /**
365    * Simple class to hold a node path and node data.
366    */
367   public static class NodeAndData {
368     private String node;
369     private byte [] data;
370     public NodeAndData(String node, byte [] data) {
371       this.node = node;
372       this.data = data;
373     }
374     public String getNode() {
375       return node;
376     }
377     public byte [] getData() {
378       return data;
379     }
380     @Override
381     public String toString() {
382       return node + " (" + RegionTransitionData.fromBytes(data) + ")";
383     }
384     public boolean isEmpty() {
385       return (data.length == 0);
386     }
387   }
388 
389   /**
390    * Checks if the specified znode has any children.  Sets no watches.
391    *
392    * Returns true if the node exists and has children.  Returns false if the
393    * node does not exist or if the node does not have any children.
394    *
395    * Used during master initialization to determine if the master is a
396    * failed-over-to master or the first master during initial cluster startup.
397    * If the directory for regionserver ephemeral nodes is empty then this is
398    * a cluster startup, if not then it is not cluster startup.
399    *
400    * @param zkw zk reference
401    * @param znode path of node to check for children of
402    * @return true if node has children, false if not or node does not exist
403    * @throws KeeperException if unexpected zookeeper exception
404    */
405   public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode)
406   throws KeeperException {
407     try {
408       return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
409     } catch(KeeperException.NoNodeException ke) {
410       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
411       "because node does not exist (not an error)"));
412       return false;
413     } catch (KeeperException e) {
414       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
415       zkw.keeperException(e);
416       return false;
417     } catch (InterruptedException e) {
418       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
419       zkw.interruptedException(e);
420       return false;
421     }
422   }
423 
424   /**
425    * Get the number of children of the specified node.
426    *
427    * If the node does not exist or has no children, returns 0.
428    *
429    * Sets no watches at all.
430    *
431    * @param zkw zk reference
432    * @param znode path of node to count children of
433    * @return number of children of specified node, 0 if none or parent does not
434    *         exist
435    * @throws KeeperException if unexpected zookeeper exception
436    */
437   public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode)
438   throws KeeperException {
439     try {
440       Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
441       return stat == null ? 0 : stat.getNumChildren();
442     } catch(KeeperException e) {
443       LOG.warn(zkw.prefix("Unable to get children of node " + znode));
444       zkw.keeperException(e);
445     } catch(InterruptedException e) {
446       zkw.interruptedException(e);
447     }
448     return 0;
449   }
450 
451   //
452   // Data retrieval
453   //
454 
455   /**
456    * Get znode data. Does not set a watcher.
457    * @return ZNode data
458    */
459   public static byte [] getData(ZooKeeperWatcher zkw, String znode)
460   throws KeeperException {
461     try {
462       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
463       logRetrievedMsg(zkw, znode, data, false);
464       return data;
465     } catch (KeeperException.NoNodeException e) {
466       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
467         "because node does not exist (not an error)"));
468       return null;
469     } catch (KeeperException e) {
470       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
471       zkw.keeperException(e);
472       return null;
473     } catch (InterruptedException e) {
474       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
475       zkw.interruptedException(e);
476       return null;
477     }
478   }
479 
480   /**
481    * Get the data at the specified znode and set a watch.
482    *
483    * Returns the data and sets a watch if the node exists.  Returns null and no
484    * watch is set if the node does not exist or there is an exception.
485    *
486    * @param zkw zk reference
487    * @param znode path of node
488    * @return data of the specified znode, or null
489    * @throws KeeperException if unexpected zookeeper exception
490    */
491   public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
492   throws KeeperException {
493     return getDataInternal(zkw, znode, null, true);
494   }
495 
496   /**
497    * Get the data at the specified znode and set a watch.
498    *
499    * Returns the data and sets a watch if the node exists.  Returns null and no
500    * watch is set if the node does not exist or there is an exception.
501    *
502    * @param zkw zk reference
503    * @param znode path of node
504    * @param stat object to populate the version of the znode
505    * @return data of the specified znode, or null
506    * @throws KeeperException if unexpected zookeeper exception
507    */
508   public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
509       Stat stat) throws KeeperException {
510     return getDataInternal(zkw, znode, stat, true);
511   }
512 
513   private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
514       boolean watcherSet)
515       throws KeeperException {
516     try {
517       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
518       logRetrievedMsg(zkw, znode, data, watcherSet);
519       return data;
520     } catch (KeeperException.NoNodeException e) {
521       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
522         "because node does not exist (not an error)"));
523       return null;
524     } catch (KeeperException e) {
525       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
526       zkw.keeperException(e);
527       return null;
528     } catch (InterruptedException e) {
529       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
530       zkw.interruptedException(e);
531       return null;
532     }
533   }
534 
535   /**
536    * Get the data at the specified znode without setting a watch.
537    *
538    * Returns the data if the node exists.  Returns null if the node does not
539    * exist.
540    *
541    * Sets the stats of the node in the passed Stat object.  Pass a null stat if
542    * not interested.
543    *
544    * @param zkw zk reference
545    * @param znode path of node
546    * @param stat node status to get if node exists
547    * @return data of the specified znode, or null if node does not exist
548    * @throws KeeperException if unexpected zookeeper exception
549    */
550   public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode,
551       Stat stat)
552   throws KeeperException {
553     try {
554       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
555       logRetrievedMsg(zkw, znode, data, false);
556       return data;
557     } catch (KeeperException.NoNodeException e) {
558       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
559           "because node does not exist (not necessarily an error)"));
560       return null;
561     } catch (KeeperException e) {
562       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
563       zkw.keeperException(e);
564       return null;
565     } catch (InterruptedException e) {
566       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
567       zkw.interruptedException(e);
568       return null;
569     }
570   }
571 
572   /**
573    * Returns the date of child znodes of the specified znode.  Also sets a watch on
574    * the specified znode which will capture a NodeDeleted event on the specified
575    * znode as well as NodeChildrenChanged if any children of the specified znode
576    * are created or deleted.
577    *
578    * Returns null if the specified node does not exist.  Otherwise returns a
579    * list of children of the specified node.  If the node exists but it has no
580    * children, an empty list will be returned.
581    *
582    * @param zkw zk reference
583    * @param baseNode path of node to list and watch children of
584    * @return list of data of children of the specified node, an empty list if the node
585    *          exists but has no children, and null if the node does not exist
586    * @throws KeeperException if unexpected zookeeper exception
587    */
588   public static List<NodeAndData> getChildDataAndWatchForNewChildren(
589       ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
590     List<String> nodes =
591       ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
592     List<NodeAndData> newNodes = new ArrayList<NodeAndData>();
593     for (String node: nodes) {
594       String nodePath = ZKUtil.joinZNode(baseNode, node);
595       byte [] data = ZKUtil.getDataAndWatch(zkw, nodePath);
596       newNodes.add(new NodeAndData(nodePath, data));
597     }
598     return newNodes;
599   }
600 
601   /**
602    * Update the data of an existing node with the expected version to have the
603    * specified data.
604    *
605    * Throws an exception if there is a version mismatch or some other problem.
606    *
607    * Sets no watches under any conditions.
608    *
609    * @param zkw zk reference
610    * @param znode
611    * @param data
612    * @param expectedVersion
613    * @throws KeeperException if unexpected zookeeper exception
614    * @throws KeeperException.BadVersionException if version mismatch
615    */
616   public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode,
617       byte [] data, int expectedVersion)
618   throws KeeperException {
619     try {
620       zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
621     } catch(InterruptedException ie) {
622       zkw.interruptedException(ie);
623     }
624   }
625 
626   //
627   // Data setting
628   //
629 
630   /**
631    * Sets the data of the existing znode to be the specified data.  Ensures that
632    * the current data has the specified expected version.
633    *
634    * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
635    *
636    * <p>If their is a version mismatch, method returns null.
637    *
638    * <p>No watches are set but setting data will trigger other watchers of this
639    * node.
640    *
641    * <p>If there is another problem, a KeeperException will be thrown.
642    *
643    * @param zkw zk reference
644    * @param znode path of node
645    * @param data data to set for node
646    * @param expectedVersion version expected when setting data
647    * @return true if data set, false if version mismatch
648    * @throws KeeperException if unexpected zookeeper exception
649    */
650   public static boolean setData(ZooKeeperWatcher zkw, String znode,
651       byte [] data, int expectedVersion)
652   throws KeeperException, KeeperException.NoNodeException {
653     try {
654       return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
655     } catch (InterruptedException e) {
656       zkw.interruptedException(e);
657       return false;
658     }
659   }
660 
661   /**
662    * Set data into node creating node if it doesn't yet exist.
663    * Does not set watch.
664    * @param zkw zk reference
665    * @param znode path of node
666    * @param data data to set for node
667    * @throws KeeperException
668    */
669   public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
670       final byte [] data)
671   throws KeeperException {
672     if (checkExists(zkw, znode) == -1) {
673       ZKUtil.createWithParents(zkw, znode);
674     }
675     ZKUtil.setData(zkw, znode, data);
676   }
677 
678   /**
679    * Sets the data of the existing znode to be the specified data.  The node
680    * must exist but no checks are done on the existing data or version.
681    *
682    * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
683    *
684    * <p>No watches are set but setting data will trigger other watchers of this
685    * node.
686    *
687    * <p>If there is another problem, a KeeperException will be thrown.
688    *
689    * @param zkw zk reference
690    * @param znode path of node
691    * @param data data to set for node
692    * @throws KeeperException if unexpected zookeeper exception
693    */
694   public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
695   throws KeeperException, KeeperException.NoNodeException {
696     setData(zkw, znode, data, -1);
697   }
698 
699   public static boolean isSecureZooKeeper(Configuration conf) {
700     // TODO: We need a better check for security enabled ZooKeeper. Currently
701     // the secure ZooKeeper client is set up using a supplied JaaS
702     // configuration file. But if the system property for the JaaS
703     // configuration file is set, this may not be an exclusive indication
704     // that HBase should set ACLs on znodes. As an alternative, we could do
705     // this more like Hadoop and build a JaaS configuration programmatically
706     // based on a site conf setting. The scope of such a change will be
707     // addressed in HBASE-4791.
708     return (System.getProperty("java.security.auth.login.config") != null);
709   }
710 
711   private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
712     if (isSecureZooKeeper(zkw.getConfiguration())) {
713       // Certain znodes must be readable by non-authenticated clients
714       if ((node.equals(zkw.rootServerZNode) == true) ||
715           (node.equals(zkw.masterAddressZNode) == true) ||
716           (node.equals(zkw.clusterIdZNode) == true)) {
717         return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
718       }
719       return Ids.CREATOR_ALL_ACL;
720     } else {
721       return Ids.OPEN_ACL_UNSAFE;
722     }
723   }
724 
725   public static void waitForZKConnectionIfAuthenticating(ZooKeeperWatcher zkw)
726       throws InterruptedException {
727     if (isSecureZooKeeper(zkw.getConfiguration())) {
728        LOG.debug("Waiting for ZooKeeperWatcher to authenticate");
729        zkw.saslLatch.await();
730        LOG.debug("Done waiting.");
731     }
732   }
733 
734   //
735   // Node creation
736   //
737 
738   /**
739    *
740    * Set the specified znode to be an ephemeral node carrying the specified
741    * data.
742    *
743    * If the node is created successfully, a watcher is also set on the node.
744    *
745    * If the node is not created successfully because it already exists, this
746    * method will also set a watcher on the node.
747    *
748    * If there is another problem, a KeeperException will be thrown.
749    *
750    * @param zkw zk reference
751    * @param znode path of node
752    * @param data data of node
753    * @return true if node created, false if not, watch set in both cases
754    * @throws KeeperException if unexpected zookeeper exception
755    */
756   public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
757       String znode, byte [] data)
758   throws KeeperException {
759     try {
760       waitForZKConnectionIfAuthenticating(zkw);
761       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
762           CreateMode.EPHEMERAL);
763     } catch (KeeperException.NodeExistsException nee) {
764       if(!watchAndCheckExists(zkw, znode)) {
765         // It did exist but now it doesn't, try again
766         return createEphemeralNodeAndWatch(zkw, znode, data);
767       }
768       return false;
769     } catch (InterruptedException e) {
770       LOG.info("Interrupted", e);
771       Thread.currentThread().interrupt();
772     }
773     return true;
774   }
775 
776   /**
777    * Creates the specified znode to be a persistent node carrying the specified
778    * data.
779    *
780    * Returns true if the node was successfully created, false if the node
781    * already existed.
782    *
783    * If the node is created successfully, a watcher is also set on the node.
784    *
785    * If the node is not created successfully because it already exists, this
786    * method will also set a watcher on the node but return false.
787    *
788    * If there is another problem, a KeeperException will be thrown.
789    *
790    * @param zkw zk reference
791    * @param znode path of node
792    * @param data data of node
793    * @return true if node created, false if not, watch set in both cases
794    * @throws KeeperException if unexpected zookeeper exception
795    */
796   public static boolean createNodeIfNotExistsAndWatch(
797       ZooKeeperWatcher zkw, String znode, byte [] data)
798   throws KeeperException {
799     try {
800       waitForZKConnectionIfAuthenticating(zkw);
801       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
802           CreateMode.PERSISTENT);
803     } catch (KeeperException.NodeExistsException nee) {
804       try {
805         zkw.getRecoverableZooKeeper().exists(znode, zkw);
806       } catch (InterruptedException e) {
807         zkw.interruptedException(e);
808         return false;
809       }
810       return false;
811     } catch (InterruptedException e) {
812       zkw.interruptedException(e);
813       return false;
814     }
815     return true;
816   }
817 
818   /**
819    * Creates the specified node with the specified data and watches it.
820    *
821    * <p>Throws an exception if the node already exists.
822    *
823    * <p>The node created is persistent and open access.
824    *
825    * <p>Returns the version number of the created node if successful.
826    *
827    * @param zkw zk reference
828    * @param znode path of node to create
829    * @param data data of node to create
830    * @return version of node created
831    * @throws KeeperException if unexpected zookeeper exception
832    * @throws KeeperException.NodeExistsException if node already exists
833    */
834   public static int createAndWatch(ZooKeeperWatcher zkw,
835       String znode, byte [] data)
836   throws KeeperException, KeeperException.NodeExistsException {
837     try {
838       waitForZKConnectionIfAuthenticating(zkw);
839       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
840           CreateMode.PERSISTENT);
841       return zkw.getRecoverableZooKeeper().exists(znode, zkw).getVersion();
842     } catch (InterruptedException e) {
843       zkw.interruptedException(e);
844       return -1;
845     }
846   }
847 
848   /**
849    * Async creates the specified node with the specified data.
850    *
851    * <p>Throws an exception if the node already exists.
852    *
853    * <p>The node created is persistent and open access.
854    *
855    * @param zkw zk reference
856    * @param znode path of node to create
857    * @param data data of node to create
858    * @param cb
859    * @param ctx
860    * @throws KeeperException if unexpected zookeeper exception
861    * @throws KeeperException.NodeExistsException if node already exists
862    */
863   public static void asyncCreate(ZooKeeperWatcher zkw,
864       String znode, byte [] data, final AsyncCallback.StringCallback cb,
865       final Object ctx) {
866     try {
867       waitForZKConnectionIfAuthenticating(zkw);
868       zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
869           createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
870     } catch (InterruptedException e) {
871       zkw.interruptedException(e);
872     }
873   }
874 
875   /**
876    * Creates the specified node, if the node does not exist.  Does not set a
877    * watch and fails silently if the node already exists.
878    *
879    * The node created is persistent and open access.
880    *
881    * @param zkw zk reference
882    * @param znode path of node
883    * @throws KeeperException if unexpected zookeeper exception
884    */
885   public static void createAndFailSilent(ZooKeeperWatcher zkw,
886       String znode)
887   throws KeeperException {
888     try {
889       RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
890       waitForZKConnectionIfAuthenticating(zkw);
891       if (zk.exists(znode, false) == null) {
892         zk.create(znode, new byte[0], createACL(zkw,znode),
893             CreateMode.PERSISTENT);
894       }
895     } catch(KeeperException.NodeExistsException nee) {
896     } catch(KeeperException.NoAuthException nee){
897       try {
898         if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
899           // If we failed to create the file and it does not already exist.
900           throw(nee);
901         }
902       } catch (InterruptedException ie) {
903         zkw.interruptedException(ie);
904       }
905 
906     } catch(InterruptedException ie) {
907       zkw.interruptedException(ie);
908     }
909   }
910 
911   /**
912    * Creates the specified node and all parent nodes required for it to exist.
913    *
914    * No watches are set and no errors are thrown if the node already exists.
915    *
916    * The nodes created are persistent and open access.
917    *
918    * @param zkw zk reference
919    * @param znode path of node
920    * @throws KeeperException if unexpected zookeeper exception
921    */
922   public static void createWithParents(ZooKeeperWatcher zkw, String znode)
923   throws KeeperException {
924     try {
925       if(znode == null) {
926         return;
927       }
928       waitForZKConnectionIfAuthenticating(zkw);
929       zkw.getRecoverableZooKeeper().create(znode, new byte[0], createACL(zkw, znode),
930           CreateMode.PERSISTENT);
931     } catch(KeeperException.NodeExistsException nee) {
932       return;
933     } catch(KeeperException.NoNodeException nne) {
934       createWithParents(zkw, getParent(znode));
935       createWithParents(zkw, znode);
936     } catch(InterruptedException ie) {
937       zkw.interruptedException(ie);
938     }
939   }
940 
941   //
942   // Deletes
943   //
944 
945   /**
946    * Delete the specified node.  Sets no watches.  Throws all exceptions.
947    */
948   public static void deleteNode(ZooKeeperWatcher zkw, String node)
949   throws KeeperException {
950     deleteNode(zkw, node, -1);
951   }
952 
953   /**
954    * Delete the specified node with the specified version.  Sets no watches.
955    * Throws all exceptions.
956    */
957   public static boolean deleteNode(ZooKeeperWatcher zkw, String node,
958       int version)
959   throws KeeperException {
960     try {
961       zkw.getRecoverableZooKeeper().delete(node, version);
962       return true;
963     } catch(KeeperException.BadVersionException bve) {
964       return false;
965     } catch(InterruptedException ie) {
966       zkw.interruptedException(ie);
967       return false;
968     }
969   }
970 
971   /**
972    * Deletes the specified node.  Fails silent if the node does not exist.
973    * @param zkw
974    * @param node
975    * @throws KeeperException
976    */
977   public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
978   throws KeeperException {
979     try {
980       zkw.getRecoverableZooKeeper().delete(node, -1);
981     } catch(KeeperException.NoNodeException nne) {
982     } catch(InterruptedException ie) {
983       zkw.interruptedException(ie);
984     }
985   }
986 
987   /**
988    * Delete the specified node and all of it's children.
989    *
990    * Sets no watches.  Throws all exceptions besides dealing with deletion of
991    * children.
992    */
993   public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node)
994   throws KeeperException {
995     try {
996       List<String> children = ZKUtil.listChildrenNoWatch(zkw, node);
997       if(!children.isEmpty()) {
998         for(String child : children) {
999           deleteNodeRecursively(zkw, joinZNode(node, child));
1000         }
1001       }
1002       zkw.getRecoverableZooKeeper().delete(node, -1);
1003     } catch(InterruptedException ie) {
1004       zkw.interruptedException(ie);
1005     }
1006   }
1007 
1008   /**
1009    * Delete all the children of the specified node but not the node itself.
1010    *
1011    * Sets no watches.  Throws all exceptions besides dealing with deletion of
1012    * children.
1013    */
1014   public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
1015   throws KeeperException {
1016     List<String> children = ZKUtil.listChildrenNoWatch(zkw, node);
1017     if (children == null || children.isEmpty()) return;
1018     for(String child : children) {
1019       deleteNodeRecursively(zkw, joinZNode(node, child));
1020     }
1021   }
1022 
1023   //
1024   // ZooKeeper cluster information
1025   //
1026 
1027   /** @return String dump of everything in ZooKeeper. */
1028   public static String dump(ZooKeeperWatcher zkw) {
1029     StringBuilder sb = new StringBuilder();
1030     try {
1031       sb.append("HBase is rooted at ").append(zkw.baseZNode);
1032       sb.append("\nMaster address: ").append(
1033           Bytes.toStringBinary(getData(zkw, zkw.masterAddressZNode)));
1034       sb.append("\nRegion server holding ROOT: ").append(
1035           Bytes.toStringBinary(getData(zkw, zkw.rootServerZNode)));
1036       sb.append("\nRegion servers:");
1037       for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
1038         sb.append("\n ").append(child);
1039       }
1040       sb.append("\nQuorum Server Statistics:");
1041       String[] servers = zkw.getQuorum().split(",");
1042       for (String server : servers) {
1043         sb.append("\n ").append(server);
1044         try {
1045           String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
1046 
1047           if (stat == null) {
1048             sb.append("[Error] invalid quorum server: " + server);
1049             break;
1050           }
1051 
1052           for (String s : stat) {
1053             sb.append("\n  ").append(s);
1054           }
1055         } catch (Exception e) {
1056           sb.append("\n  ERROR: ").append(e.getMessage());
1057         }
1058       }
1059     } catch (KeeperException ke) {
1060       sb.append("\nFATAL ZooKeeper Exception!\n");
1061       sb.append("\n" + ke.getMessage());
1062     }
1063     return sb.toString();
1064   }
1065 
1066   /**
1067    * Gets the statistics from the given server.
1068    *
1069    * @param server  The server to get the statistics from.
1070    * @param timeout  The socket timeout to use.
1071    * @return The array of response strings.
1072    * @throws IOException When the socket communication fails.
1073    */
1074   public static String[] getServerStats(String server, int timeout)
1075   throws IOException {
1076     String[] sp = server.split(":");
1077     if (sp == null || sp.length == 0) {
1078       return null;
1079     }
1080 
1081     String host = sp[0];
1082     int port = sp.length > 1 ? Integer.parseInt(sp[1])
1083         : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
1084 
1085     Socket socket = new Socket();
1086     InetSocketAddress sockAddr = new InetSocketAddress(host, port);
1087     socket.connect(sockAddr, timeout);
1088 
1089     socket.setSoTimeout(timeout);
1090     PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
1091     BufferedReader in = new BufferedReader(new InputStreamReader(
1092       socket.getInputStream()));
1093     out.println("stat");
1094     out.flush();
1095     ArrayList<String> res = new ArrayList<String>();
1096     while (true) {
1097       String line = in.readLine();
1098       if (line != null) {
1099         res.add(line);
1100       } else {
1101         break;
1102       }
1103     }
1104     socket.close();
1105     return res.toArray(new String[res.size()]);
1106   }
1107 
1108   private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
1109       final String znode, final byte [] data, final boolean watcherSet) {
1110     if (!LOG.isDebugEnabled()) return;
1111     LOG.debug(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
1112       " byte(s) of data from znode " + znode +
1113       (watcherSet? " and set watcher; ": "; data=") +
1114       (data == null? "null": data.length == 0? "empty": (
1115           znode.startsWith(zkw.assignmentZNode) ?
1116               RegionTransitionData.fromBytes(data).toString()
1117               : StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
1118   }
1119 }