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.BufferedReader;
22  import java.io.IOException;
23  import java.io.InputStreamReader;
24  import java.io.PrintWriter;
25  import java.net.InetSocketAddress;
26  import java.net.Socket;
27  import java.util.ArrayList;
28  import java.util.Arrays;
29  import java.util.HashMap;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Properties;
34  
35  import javax.security.auth.login.AppConfigurationEntry;
36  import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag;
37  
38  import org.apache.commons.lang.StringUtils;
39  import org.apache.commons.logging.Log;
40  import org.apache.commons.logging.LogFactory;
41  import org.apache.hadoop.conf.Configuration;
42  import org.apache.hadoop.hbase.HConstants;
43  import org.apache.hadoop.hbase.ServerName;
44  import org.apache.hadoop.hbase.classification.InterfaceAudience;
45  import org.apache.hadoop.hbase.exceptions.DeserializationException;
46  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
47  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
48  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
49  import org.apache.hadoop.hbase.util.ByteStringer;
50  import org.apache.hadoop.hbase.util.Bytes;
51  import org.apache.hadoop.hbase.util.Threads;
52  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
53  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
54  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
55  import org.apache.hadoop.security.SecurityUtil;
56  import org.apache.hadoop.security.authentication.util.KerberosUtil;
57  import org.apache.zookeeper.AsyncCallback;
58  import org.apache.zookeeper.CreateMode;
59  import org.apache.zookeeper.KeeperException;
60  import org.apache.zookeeper.KeeperException.NoNodeException;
61  import org.apache.zookeeper.Op;
62  import org.apache.zookeeper.Watcher;
63  import org.apache.zookeeper.ZooDefs.Ids;
64  import org.apache.zookeeper.ZooKeeper;
65  import org.apache.zookeeper.client.ZooKeeperSaslClient;
66  import org.apache.zookeeper.data.ACL;
67  import org.apache.zookeeper.data.Stat;
68  import org.apache.zookeeper.proto.CreateRequest;
69  import org.apache.zookeeper.proto.DeleteRequest;
70  import org.apache.zookeeper.proto.SetDataRequest;
71  import org.apache.zookeeper.server.ZooKeeperSaslServer;
72  
73  import com.google.protobuf.InvalidProtocolBufferException;
74  
75  /**
76   * Internal HBase utility class for ZooKeeper.
77   *
78   * <p>Contains only static methods and constants.
79   *
80   * <p>Methods all throw {@link KeeperException} if there is an unexpected
81   * zookeeper exception, so callers of these methods must handle appropriately.
82   * If ZK is required for the operation, the server will need to be aborted.
83   */
84  @InterfaceAudience.Private
85  public class ZKUtil {
86    private static final Log LOG = LogFactory.getLog(ZKUtil.class);
87  
88    // TODO: Replace this with ZooKeeper constant when ZOOKEEPER-277 is resolved.
89    public static final char ZNODE_PATH_SEPARATOR = '/';
90    private static int zkDumpConnectionTimeOut;
91  
92    /**
93     * Creates a new connection to ZooKeeper, pulling settings and ensemble config
94     * from the specified configuration object using methods from {@link ZKConfig}.
95     *
96     * Sets the connection status monitoring watcher to the specified watcher.
97     *
98     * @param conf configuration to pull ensemble and other settings from
99     * @param watcher watcher to monitor connection changes
100    * @return connection to zookeeper
101    * @throws IOException if unable to connect to zk or config problem
102    */
103   public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
104   throws IOException {
105     Properties properties = ZKConfig.makeZKProps(conf);
106     String ensemble = ZKConfig.getZKQuorumServersString(properties);
107     return connect(conf, ensemble, watcher);
108   }
109 
110   public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
111       Watcher watcher)
112   throws IOException {
113     return connect(conf, ensemble, watcher, null);
114   }
115 
116   public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
117       Watcher watcher, final String identifier)
118   throws IOException {
119     if(ensemble == null) {
120       throw new IOException("Unable to determine ZooKeeper ensemble");
121     }
122     int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
123         HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
124     if (LOG.isTraceEnabled()) {
125       LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
126     }
127     int retry = conf.getInt("zookeeper.recovery.retry", 3);
128     int retryIntervalMillis =
129       conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
130     zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
131         1000);
132     return new RecoverableZooKeeper(ensemble, timeout, watcher,
133         retry, retryIntervalMillis, identifier);
134   }
135 
136   /**
137    * Log in the current zookeeper server process using the given configuration
138    * keys for the credential file and login principal.
139    *
140    * <p><strong>This is only applicable when running on secure hbase</strong>
141    * On regular HBase (without security features), this will safely be ignored.
142    * </p>
143    *
144    * @param conf The configuration data to use
145    * @param keytabFileKey Property key used to configure the path to the credential file
146    * @param userNameKey Property key used to configure the login principal
147    * @param hostname Current hostname to use in any credentials
148    * @throws IOException underlying exception from SecurityUtil.login() call
149    */
150   public static void loginServer(Configuration conf, String keytabFileKey,
151       String userNameKey, String hostname) throws IOException {
152     login(conf, keytabFileKey, userNameKey, hostname,
153           ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
154           JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
155   }
156 
157   /**
158    * Log in the current zookeeper client using the given configuration
159    * keys for the credential file and login principal.
160    *
161    * <p><strong>This is only applicable when running on secure hbase</strong>
162    * On regular HBase (without security features), this will safely be ignored.
163    * </p>
164    *
165    * @param conf The configuration data to use
166    * @param keytabFileKey Property key used to configure the path to the credential file
167    * @param userNameKey Property key used to configure the login principal
168    * @param hostname Current hostname to use in any credentials
169    * @throws IOException underlying exception from SecurityUtil.login() call
170    */
171   public static void loginClient(Configuration conf, String keytabFileKey,
172       String userNameKey, String hostname) throws IOException {
173     login(conf, keytabFileKey, userNameKey, hostname,
174           ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
175           JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
176   }
177 
178   /**
179    * Log in the current process using the given configuration keys for the
180    * credential file and login principal.
181    *
182    * <p><strong>This is only applicable when running on secure hbase</strong>
183    * On regular HBase (without security features), this will safely be ignored.
184    * </p>
185    *
186    * @param conf The configuration data to use
187    * @param keytabFileKey Property key used to configure the path to the credential file
188    * @param userNameKey Property key used to configure the login principal
189    * @param hostname Current hostname to use in any credentials
190    * @param loginContextProperty property name to expose the entry name
191    * @param loginContextName jaas entry name
192    * @throws IOException underlying exception from SecurityUtil.login() call
193    */
194   private static void login(Configuration conf, String keytabFileKey,
195       String userNameKey, String hostname,
196       String loginContextProperty, String loginContextName)
197       throws IOException {
198     if (!isSecureZooKeeper(conf))
199       return;
200 
201     // User has specified a jaas.conf, keep this one as the good one.
202     // HBASE_OPTS="-Djava.security.auth.login.config=jaas.conf"
203     if (System.getProperty("java.security.auth.login.config") != null)
204       return;
205 
206     // No keytab specified, no auth
207     String keytabFilename = conf.get(keytabFileKey);
208     if (keytabFilename == null) {
209       LOG.warn("no keytab specified for: " + keytabFileKey);
210       return;
211     }
212 
213     String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
214     String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
215 
216     // Initialize the "jaas.conf" for keyTab/principal,
217     // If keyTab is not specified use the Ticket Cache.
218     // and set the zookeeper login context name.
219     JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
220         principalName, keytabFilename);
221     javax.security.auth.login.Configuration.setConfiguration(jaasConf);
222     System.setProperty(loginContextProperty, loginContextName);
223   }
224 
225   /**
226    * A JAAS configuration that defines the login modules that we want to use for login.
227    */
228   private static class JaasConfiguration extends javax.security.auth.login.Configuration {
229     private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
230       "zookeeper-server-keytab-kerberos";
231     private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
232       "zookeeper-client-keytab-kerberos";
233 
234     private static final Map<String, String> BASIC_JAAS_OPTIONS =
235       new HashMap<String,String>();
236     static {
237       String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
238       if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
239         BASIC_JAAS_OPTIONS.put("debug", "true");
240       }
241     }
242 
243     private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS =
244       new HashMap<String,String>();
245     static {
246       KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
247       KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
248       KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
249       KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
250     }
251 
252     private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
253       new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
254                                 LoginModuleControlFlag.REQUIRED,
255                                 KEYTAB_KERBEROS_OPTIONS);
256 
257     private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
258       new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
259 
260     private javax.security.auth.login.Configuration baseConfig;
261     private final String loginContextName;
262     private final boolean useTicketCache;
263     private final String keytabFile;
264     private final String principal;
265 
266     public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
267       this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
268     }
269 
270     private JaasConfiguration(String loginContextName, String principal,
271                              String keytabFile, boolean useTicketCache) {
272       try {
273         this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
274       } catch (SecurityException e) {
275         this.baseConfig = null;
276       }
277       this.loginContextName = loginContextName;
278       this.useTicketCache = useTicketCache;
279       this.keytabFile = keytabFile;
280       this.principal = principal;
281       LOG.info("JaasConfiguration loginContextName=" + loginContextName +
282                " principal=" + principal + " useTicketCache=" + useTicketCache +
283                " keytabFile=" + keytabFile);
284     }
285 
286     @Override
287     public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
288       if (loginContextName.equals(appName)) {
289         if (!useTicketCache) {
290           KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
291           KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
292         }
293         KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
294         KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
295         return KEYTAB_KERBEROS_CONF;
296       }
297       if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
298       return(null);
299     }
300   }
301 
302   //
303   // Helper methods
304   //
305 
306   /**
307    * Join the prefix znode name with the suffix znode name to generate a proper
308    * full znode name.
309    *
310    * Assumes prefix does not end with slash and suffix does not begin with it.
311    *
312    * @param prefix beginning of znode name
313    * @param suffix ending of znode name
314    * @return result of properly joining prefix with suffix
315    */
316   public static String joinZNode(String prefix, String suffix) {
317     return prefix + ZNODE_PATH_SEPARATOR + suffix;
318   }
319 
320   /**
321    * Returns the full path of the immediate parent of the specified node.
322    * @param node path to get parent of
323    * @return parent of path, null if passed the root node or an invalid node
324    */
325   public static String getParent(String node) {
326     int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR);
327     return idx <= 0 ? null : node.substring(0, idx);
328   }
329 
330   /**
331    * Get the name of the current node from the specified fully-qualified path.
332    * @param path fully-qualified path
333    * @return name of the current node
334    */
335   public static String getNodeName(String path) {
336     return path.substring(path.lastIndexOf("/")+1);
337   }
338 
339   /**
340    * Get the key to the ZK ensemble for this configuration without
341    * adding a name at the end
342    * @param conf Configuration to use to build the key
343    * @return ensemble key without a name
344    */
345   public static String getZooKeeperClusterKey(Configuration conf) {
346     return getZooKeeperClusterKey(conf, null);
347   }
348 
349   /**
350    * Get the key to the ZK ensemble for this configuration and append
351    * a name at the end
352    * @param conf Configuration to use to build the key
353    * @param name Name that should be appended at the end if not empty or null
354    * @return ensemble key with a name (if any)
355    */
356   public static String getZooKeeperClusterKey(Configuration conf, String name) {
357     String ensemble = conf.get(HConstants.ZOOKEEPER_QUORUM.replaceAll(
358         "[\\t\\n\\x0B\\f\\r]", ""));
359     StringBuilder builder = new StringBuilder(ensemble);
360     builder.append(":");
361     builder.append(conf.get(HConstants.ZOOKEEPER_CLIENT_PORT));
362     builder.append(":");
363     builder.append(conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT));
364     if (name != null && !name.isEmpty()) {
365       builder.append(",");
366       builder.append(name);
367     }
368     return builder.toString();
369   }
370 
371   /**
372    * Apply the settings in the given key to the given configuration, this is
373    * used to communicate with distant clusters
374    * @param conf configuration object to configure
375    * @param key string that contains the 3 required configuratins
376    * @throws IOException
377    */
378   public static void applyClusterKeyToConf(Configuration conf, String key)
379       throws IOException{
380     String[] parts = transformClusterKey(key);
381     conf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
382     conf.set(HConstants.ZOOKEEPER_CLIENT_PORT, parts[1]);
383     conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[2]);
384   }
385 
386   /**
387    * Separate the given key into the three configurations it should contain:
388    * hbase.zookeeper.quorum, hbase.zookeeper.client.port
389    * and zookeeper.znode.parent
390    * @param key
391    * @return the three configuration in the described order
392    * @throws IOException
393    */
394   public static String[] transformClusterKey(String key) throws IOException {
395     String[] parts = key.split(":");
396     if (parts.length != 3) {
397       throw new IOException("Cluster key passed " + key + " is invalid, the format should be:" +
398           HConstants.ZOOKEEPER_QUORUM + ":hbase.zookeeper.client.port:"
399           + HConstants.ZOOKEEPER_ZNODE_PARENT);
400     }
401     return parts;
402   }
403 
404   //
405   // Existence checks and watches
406   //
407 
408   /**
409    * Watch the specified znode for delete/create/change events.  The watcher is
410    * set whether or not the node exists.  If the node already exists, the method
411    * returns true.  If the node does not exist, the method returns false.
412    *
413    * @param zkw zk reference
414    * @param znode path of node to watch
415    * @return true if znode exists, false if does not exist or error
416    * @throws KeeperException if unexpected zookeeper exception
417    */
418   public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode)
419   throws KeeperException {
420     try {
421       Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
422       boolean exists = s != null ? true : false;
423       if (exists) {
424         LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
425       } else {
426         LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
427       }
428       return exists;
429     } catch (KeeperException e) {
430       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
431       zkw.keeperException(e);
432       return false;
433     } catch (InterruptedException e) {
434       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
435       zkw.interruptedException(e);
436       return false;
437     }
438   }
439 
440   /**
441    * Watch the specified znode, but only if exists. Useful when watching
442    * for deletions. Uses .getData() (and handles NoNodeException) instead
443    * of .exists() to accomplish this, as .getData() will only set a watch if
444    * the znode exists.
445    * @param zkw zk reference
446    * @param znode path of node to watch
447    * @return true if the watch is set, false if node does not exists
448    * @throws KeeperException if unexpected zookeeper exception
449    */
450   public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode)
451       throws KeeperException {
452     try {
453       zkw.getRecoverableZooKeeper().getData(znode, true, null);
454       return true;
455     } catch (NoNodeException e) {
456       return false;
457     } catch (InterruptedException e) {
458       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
459       zkw.interruptedException(e);
460       return false;
461     }
462   }
463 
464   /**
465    * Check if the specified node exists.  Sets no watches.
466    *
467    * @param zkw zk reference
468    * @param znode path of node to watch
469    * @return version of the node if it exists, -1 if does not exist
470    * @throws KeeperException if unexpected zookeeper exception
471    */
472   public static int checkExists(ZooKeeperWatcher zkw, String znode)
473   throws KeeperException {
474     try {
475       Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
476       return s != null ? s.getVersion() : -1;
477     } catch (KeeperException e) {
478       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
479       zkw.keeperException(e);
480       return -1;
481     } catch (InterruptedException e) {
482       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
483       zkw.interruptedException(e);
484       return -1;
485     }
486   }
487 
488   //
489   // Znode listings
490   //
491 
492   /**
493    * Lists the children znodes of the specified znode.  Also sets a watch on
494    * the specified znode which will capture a NodeDeleted event on the specified
495    * znode as well as NodeChildrenChanged if any children of the specified znode
496    * are created or deleted.
497    *
498    * Returns null if the specified node does not exist.  Otherwise returns a
499    * list of children of the specified node.  If the node exists but it has no
500    * children, an empty list will be returned.
501    *
502    * @param zkw zk reference
503    * @param znode path of node to list and watch children of
504    * @return list of children of the specified node, an empty list if the node
505    *          exists but has no children, and null if the node does not exist
506    * @throws KeeperException if unexpected zookeeper exception
507    */
508   public static List<String> listChildrenAndWatchForNewChildren(
509       ZooKeeperWatcher zkw, String znode)
510   throws KeeperException {
511     try {
512       List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
513       return children;
514     } catch(KeeperException.NoNodeException ke) {
515       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
516           "because node does not exist (not an error)"));
517       return null;
518     } catch (KeeperException e) {
519       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
520       zkw.keeperException(e);
521       return null;
522     } catch (InterruptedException e) {
523       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
524       zkw.interruptedException(e);
525       return null;
526     }
527   }
528 
529   /**
530    * List all the children of the specified znode, setting a watch for children
531    * changes and also setting a watch on every individual child in order to get
532    * the NodeCreated and NodeDeleted events.
533    * @param zkw zookeeper reference
534    * @param znode node to get children of and watch
535    * @return list of znode names, null if the node doesn't exist
536    * @throws KeeperException
537    */
538   public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
539       String znode) throws KeeperException {
540     List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
541     if (children == null) {
542       return null;
543     }
544     for (String child : children) {
545       watchAndCheckExists(zkw, joinZNode(znode, child));
546     }
547     return children;
548   }
549 
550   /**
551    * Lists the children of the specified znode without setting any watches.
552    *
553    * Sets no watches at all, this method is best effort.
554    *
555    * Returns an empty list if the node has no children.  Returns null if the
556    * parent node itself does not exist.
557    *
558    * @param zkw zookeeper reference
559    * @param znode node to get children
560    * @return list of data of children of specified znode, empty if no children,
561    *         null if parent does not exist
562    * @throws KeeperException if unexpected zookeeper exception
563    */
564   public static List<String> listChildrenNoWatch(ZooKeeperWatcher zkw, String znode)
565   throws KeeperException {
566     List<String> children = null;
567     try {
568       // List the children without watching
569       children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
570     } catch(KeeperException.NoNodeException nne) {
571       return null;
572     } catch(InterruptedException ie) {
573       zkw.interruptedException(ie);
574     }
575     return children;
576   }
577 
578   /**
579    * Simple class to hold a node path and node data.
580    * @deprecated Unused
581    */
582   @Deprecated
583   public static class NodeAndData {
584     private String node;
585     private byte [] data;
586     public NodeAndData(String node, byte [] data) {
587       this.node = node;
588       this.data = data;
589     }
590     public String getNode() {
591       return node;
592     }
593     public byte [] getData() {
594       return data;
595     }
596     @Override
597     public String toString() {
598       return node;
599     }
600     public boolean isEmpty() {
601       return (data.length == 0);
602     }
603   }
604 
605   /**
606    * Checks if the specified znode has any children.  Sets no watches.
607    *
608    * Returns true if the node exists and has children.  Returns false if the
609    * node does not exist or if the node does not have any children.
610    *
611    * Used during master initialization to determine if the master is a
612    * failed-over-to master or the first master during initial cluster startup.
613    * If the directory for regionserver ephemeral nodes is empty then this is
614    * a cluster startup, if not then it is not cluster startup.
615    *
616    * @param zkw zk reference
617    * @param znode path of node to check for children of
618    * @return true if node has children, false if not or node does not exist
619    * @throws KeeperException if unexpected zookeeper exception
620    */
621   public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode)
622   throws KeeperException {
623     try {
624       return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
625     } catch(KeeperException.NoNodeException ke) {
626       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
627       "because node does not exist (not an error)"));
628       return false;
629     } catch (KeeperException e) {
630       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
631       zkw.keeperException(e);
632       return false;
633     } catch (InterruptedException e) {
634       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
635       zkw.interruptedException(e);
636       return false;
637     }
638   }
639 
640   /**
641    * Get the number of children of the specified node.
642    *
643    * If the node does not exist or has no children, returns 0.
644    *
645    * Sets no watches at all.
646    *
647    * @param zkw zk reference
648    * @param znode path of node to count children of
649    * @return number of children of specified node, 0 if none or parent does not
650    *         exist
651    * @throws KeeperException if unexpected zookeeper exception
652    */
653   public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode)
654   throws KeeperException {
655     try {
656       Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
657       return stat == null ? 0 : stat.getNumChildren();
658     } catch(KeeperException e) {
659       LOG.warn(zkw.prefix("Unable to get children of node " + znode));
660       zkw.keeperException(e);
661     } catch(InterruptedException e) {
662       zkw.interruptedException(e);
663     }
664     return 0;
665   }
666 
667   //
668   // Data retrieval
669   //
670 
671   /**
672    * Get znode data. Does not set a watcher.
673    * @return ZNode data, null if the node does not exist or if there is an
674    *  error.
675    */
676   public static byte [] getData(ZooKeeperWatcher zkw, String znode)
677       throws KeeperException, InterruptedException {
678     try {
679       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
680       logRetrievedMsg(zkw, znode, data, false);
681       return data;
682     } catch (KeeperException.NoNodeException e) {
683       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
684           "because node does not exist (not an error)"));
685       return null;
686     } catch (KeeperException e) {
687       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
688       zkw.keeperException(e);
689       return null;
690     }
691   }
692 
693   /**
694    * Get the data at the specified znode and set a watch.
695    *
696    * Returns the data and sets a watch if the node exists.  Returns null and no
697    * watch is set if the node does not exist or there is an exception.
698    *
699    * @param zkw zk reference
700    * @param znode path of node
701    * @return data of the specified znode, or null
702    * @throws KeeperException if unexpected zookeeper exception
703    */
704   public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
705   throws KeeperException {
706     return getDataInternal(zkw, znode, null, true);
707   }
708 
709   /**
710    * Get the data at the specified znode and set a watch.
711    *
712    * Returns the data and sets a watch if the node exists.  Returns null and no
713    * watch is set if the node does not exist or there is an exception.
714    *
715    * @param zkw zk reference
716    * @param znode path of node
717    * @param stat object to populate the version of the znode
718    * @return data of the specified znode, or null
719    * @throws KeeperException if unexpected zookeeper exception
720    */
721   public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
722       Stat stat) throws KeeperException {
723     return getDataInternal(zkw, znode, stat, true);
724   }
725 
726   private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
727       boolean watcherSet)
728       throws KeeperException {
729     try {
730       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
731       logRetrievedMsg(zkw, znode, data, watcherSet);
732       return data;
733     } catch (KeeperException.NoNodeException e) {
734       // This log can get pretty annoying when we cycle on 100ms waits.
735       // Enable trace if you really want to see it.
736       LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
737         "because node does not exist (not an error)"));
738       return null;
739     } catch (KeeperException e) {
740       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
741       zkw.keeperException(e);
742       return null;
743     } catch (InterruptedException e) {
744       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
745       zkw.interruptedException(e);
746       return null;
747     }
748   }
749 
750   /**
751    * Get the data at the specified znode without setting a watch.
752    *
753    * Returns the data if the node exists.  Returns null if the node does not
754    * exist.
755    *
756    * Sets the stats of the node in the passed Stat object.  Pass a null stat if
757    * not interested.
758    *
759    * @param zkw zk reference
760    * @param znode path of node
761    * @param stat node status to get if node exists
762    * @return data of the specified znode, or null if node does not exist
763    * @throws KeeperException if unexpected zookeeper exception
764    */
765   public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode,
766       Stat stat)
767   throws KeeperException {
768     try {
769       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
770       logRetrievedMsg(zkw, znode, data, false);
771       return data;
772     } catch (KeeperException.NoNodeException e) {
773       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
774           "because node does not exist (not necessarily an error)"));
775       return null;
776     } catch (KeeperException e) {
777       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
778       zkw.keeperException(e);
779       return null;
780     } catch (InterruptedException e) {
781       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
782       zkw.interruptedException(e);
783       return null;
784     }
785   }
786 
787   /**
788    * Returns the date of child znodes of the specified znode.  Also sets a watch on
789    * the specified znode which will capture a NodeDeleted event on the specified
790    * znode as well as NodeChildrenChanged if any children of the specified znode
791    * are created or deleted.
792    *
793    * Returns null if the specified node does not exist.  Otherwise returns a
794    * list of children of the specified node.  If the node exists but it has no
795    * children, an empty list will be returned.
796    *
797    * @param zkw zk reference
798    * @param baseNode path of node to list and watch children of
799    * @return list of data of children of the specified node, an empty list if the node
800    *          exists but has no children, and null if the node does not exist
801    * @throws KeeperException if unexpected zookeeper exception
802    * @deprecated Unused
803    */
804   @Deprecated
805   public static List<NodeAndData> getChildDataAndWatchForNewChildren(
806       ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
807     List<String> nodes =
808       ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
809     if (nodes != null) {
810       List<NodeAndData> newNodes = new ArrayList<NodeAndData>();
811       for (String node : nodes) {
812         String nodePath = ZKUtil.joinZNode(baseNode, node);
813         byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
814         newNodes.add(new NodeAndData(nodePath, data));
815       }
816       return newNodes;
817     }
818     return null;
819   }
820 
821   /**
822    * Update the data of an existing node with the expected version to have the
823    * specified data.
824    *
825    * Throws an exception if there is a version mismatch or some other problem.
826    *
827    * Sets no watches under any conditions.
828    *
829    * @param zkw zk reference
830    * @param znode
831    * @param data
832    * @param expectedVersion
833    * @throws KeeperException if unexpected zookeeper exception
834    * @throws KeeperException.BadVersionException if version mismatch
835    * @deprecated Unused
836    */
837   @Deprecated
838   public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode,
839       byte [] data, int expectedVersion)
840   throws KeeperException {
841     try {
842       zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
843     } catch(InterruptedException ie) {
844       zkw.interruptedException(ie);
845     }
846   }
847 
848   //
849   // Data setting
850   //
851 
852   /**
853    * Sets the data of the existing znode to be the specified data.  Ensures that
854    * the current data has the specified expected version.
855    *
856    * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
857    *
858    * <p>If their is a version mismatch, method returns null.
859    *
860    * <p>No watches are set but setting data will trigger other watchers of this
861    * node.
862    *
863    * <p>If there is another problem, a KeeperException will be thrown.
864    *
865    * @param zkw zk reference
866    * @param znode path of node
867    * @param data data to set for node
868    * @param expectedVersion version expected when setting data
869    * @return true if data set, false if version mismatch
870    * @throws KeeperException if unexpected zookeeper exception
871    */
872   public static boolean setData(ZooKeeperWatcher zkw, String znode,
873       byte [] data, int expectedVersion)
874   throws KeeperException, KeeperException.NoNodeException {
875     try {
876       return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
877     } catch (InterruptedException e) {
878       zkw.interruptedException(e);
879       return false;
880     }
881   }
882 
883   /**
884    * Set data into node creating node if it doesn't yet exist.
885    * Does not set watch.
886    *
887    * @param zkw zk reference
888    * @param znode path of node
889    * @param data data to set for node
890    * @throws KeeperException
891    */
892   public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
893       final byte [] data)
894   throws KeeperException {
895     if (checkExists(zkw, znode) == -1) {
896       ZKUtil.createWithParents(zkw, znode, data);
897     } else {
898       ZKUtil.setData(zkw, znode, data);
899     }
900   }
901 
902   /**
903    * Sets the data of the existing znode to be the specified data.  The node
904    * must exist but no checks are done on the existing data or version.
905    *
906    * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
907    *
908    * <p>No watches are set but setting data will trigger other watchers of this
909    * node.
910    *
911    * <p>If there is another problem, a KeeperException will be thrown.
912    *
913    * @param zkw zk reference
914    * @param znode path of node
915    * @param data data to set for node
916    * @throws KeeperException if unexpected zookeeper exception
917    */
918   public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
919   throws KeeperException, KeeperException.NoNodeException {
920     setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
921   }
922 
923   private static void setData(ZooKeeperWatcher zkw, SetData setData)
924   throws KeeperException, KeeperException.NoNodeException {
925     SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
926     setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
927   }
928 
929   /**
930    * Returns whether or not secure authentication is enabled
931    * (whether <code>hbase.security.authentication</code> is set to
932    * <code>kerberos</code>.
933    */
934   public static boolean isSecureZooKeeper(Configuration conf) {
935     // Detection for embedded HBase client with jaas configuration
936     // defined for third party programs.
937     try {
938       javax.security.auth.login.Configuration testConfig = javax.security.auth.login.Configuration.getConfiguration();
939       if(testConfig.getAppConfigurationEntry("Client") == null) {
940         return false;
941       }
942     } catch(Exception e) {
943       // No Jaas configuration defined.
944       return false;
945     }
946 
947     // Master & RSs uses hbase.zookeeper.client.*
948     return("kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication")) &&
949          conf.get("hbase.zookeeper.client.keytab.file") != null);
950   }
951 
952   private static List<ACL> createACL(ZooKeeperWatcher zkw, String node) {
953     if (isSecureZooKeeper(zkw.getConfiguration())) {
954       // Certain znodes are accessed directly by the client,
955       // so they must be readable by non-authenticated clients
956       if ((node.equals(zkw.baseZNode) == true) ||
957           (node.equals(zkw.metaServerZNode) == true) ||
958           (node.equals(zkw.getMasterAddressZNode()) == true) ||
959           (node.equals(zkw.clusterIdZNode) == true) ||
960           (node.equals(zkw.rsZNode) == true) ||
961           (node.equals(zkw.backupMasterAddressesZNode) == true) ||
962           (node.startsWith(zkw.tableZNode) == true)) {
963         return ZooKeeperWatcher.CREATOR_ALL_AND_WORLD_READABLE;
964       }
965       return Ids.CREATOR_ALL_ACL;
966     } else {
967       return Ids.OPEN_ACL_UNSAFE;
968     }
969   }
970 
971   //
972   // Node creation
973   //
974 
975   /**
976    *
977    * Set the specified znode to be an ephemeral node carrying the specified
978    * data.
979    *
980    * If the node is created successfully, a watcher is also set on the node.
981    *
982    * If the node is not created successfully because it already exists, this
983    * method will also set a watcher on the node.
984    *
985    * If there is another problem, a KeeperException will be thrown.
986    *
987    * @param zkw zk reference
988    * @param znode path of node
989    * @param data data of node
990    * @return true if node created, false if not, watch set in both cases
991    * @throws KeeperException if unexpected zookeeper exception
992    */
993   public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
994       String znode, byte [] data)
995   throws KeeperException {
996     boolean ret = true;
997     try {
998       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
999           CreateMode.EPHEMERAL);
1000     } catch (KeeperException.NodeExistsException nee) {
1001       ret = false;
1002     } catch (InterruptedException e) {
1003       LOG.info("Interrupted", e);
1004       Thread.currentThread().interrupt();
1005     }
1006     if(!watchAndCheckExists(zkw, znode)) {
1007       // It did exist but now it doesn't, try again
1008       return createEphemeralNodeAndWatch(zkw, znode, data);
1009     }
1010     return ret;
1011   }
1012 
1013   /**
1014    * Creates the specified znode to be a persistent node carrying the specified
1015    * data.
1016    *
1017    * Returns true if the node was successfully created, false if the node
1018    * already existed.
1019    *
1020    * If the node is created successfully, a watcher is also set on the node.
1021    *
1022    * If the node is not created successfully because it already exists, this
1023    * method will also set a watcher on the node but return false.
1024    *
1025    * If there is another problem, a KeeperException will be thrown.
1026    *
1027    * @param zkw zk reference
1028    * @param znode path of node
1029    * @param data data of node
1030    * @return true if node created, false if not, watch set in both cases
1031    * @throws KeeperException if unexpected zookeeper exception
1032    */
1033   public static boolean createNodeIfNotExistsAndWatch(
1034       ZooKeeperWatcher zkw, String znode, byte [] data)
1035   throws KeeperException {
1036     boolean ret = true;
1037     try {
1038       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1039           CreateMode.PERSISTENT);
1040     } catch (KeeperException.NodeExistsException nee) {
1041       ret = false;
1042     } catch (InterruptedException e) {
1043       zkw.interruptedException(e);
1044       return false;
1045     }
1046     try {
1047       zkw.getRecoverableZooKeeper().exists(znode, zkw);
1048     } catch (InterruptedException e) {
1049       zkw.interruptedException(e);
1050       return false;
1051     }
1052     return ret;
1053   }
1054 
1055   /**
1056    * Creates the specified znode with the specified data but does not watch it.
1057    *
1058    * Returns the znode of the newly created node
1059    *
1060    * If there is another problem, a KeeperException will be thrown.
1061    *
1062    * @param zkw zk reference
1063    * @param znode path of node
1064    * @param data data of node
1065    * @param createMode specifying whether the node to be created is ephemeral and/or sequential
1066    * @return true name of the newly created znode or null
1067    * @throws KeeperException if unexpected zookeeper exception
1068    */
1069   public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode,
1070       byte[] data, CreateMode createMode) throws KeeperException {
1071 
1072     String createdZNode = null;
1073     try {
1074       createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
1075           createACL(zkw, znode), createMode);
1076     } catch (KeeperException.NodeExistsException nee) {
1077       return znode;
1078     } catch (InterruptedException e) {
1079       zkw.interruptedException(e);
1080       return null;
1081     }
1082     return createdZNode;
1083   }
1084 
1085   /**
1086    * Creates the specified node with the specified data and watches it.
1087    *
1088    * <p>Throws an exception if the node already exists.
1089    *
1090    * <p>The node created is persistent and open access.
1091    *
1092    * <p>Returns the version number of the created node if successful.
1093    *
1094    * @param zkw zk reference
1095    * @param znode path of node to create
1096    * @param data data of node to create
1097    * @return version of node created
1098    * @throws KeeperException if unexpected zookeeper exception
1099    * @throws KeeperException.NodeExistsException if node already exists
1100    */
1101   public static int createAndWatch(ZooKeeperWatcher zkw,
1102       String znode, byte [] data)
1103   throws KeeperException, KeeperException.NodeExistsException {
1104     try {
1105       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1106           CreateMode.PERSISTENT);
1107       Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
1108       if (stat == null){
1109         // Likely a race condition. Someone deleted the znode.
1110         throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
1111             "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
1112       }
1113      return stat.getVersion();
1114     } catch (InterruptedException e) {
1115       zkw.interruptedException(e);
1116       return -1;
1117     }
1118   }
1119 
1120   /**
1121    * Async creates the specified node with the specified data.
1122    *
1123    * <p>Throws an exception if the node already exists.
1124    *
1125    * <p>The node created is persistent and open access.
1126    *
1127    * @param zkw zk reference
1128    * @param znode path of node to create
1129    * @param data data of node to create
1130    * @param cb
1131    * @param ctx
1132    * @throws KeeperException if unexpected zookeeper exception
1133    * @throws KeeperException.NodeExistsException if node already exists
1134    */
1135   public static void asyncCreate(ZooKeeperWatcher zkw,
1136       String znode, byte [] data, final AsyncCallback.StringCallback cb,
1137       final Object ctx) {
1138     zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
1139         createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
1140   }
1141 
1142   /**
1143    * Creates the specified node, iff the node does not exist.  Does not set a
1144    * watch and fails silently if the node already exists.
1145    *
1146    * The node created is persistent and open access.
1147    *
1148    * @param zkw zk reference
1149    * @param znode path of node
1150    * @throws KeeperException if unexpected zookeeper exception
1151    */
1152   public static void createAndFailSilent(ZooKeeperWatcher zkw,
1153       String znode) throws KeeperException {
1154     createAndFailSilent(zkw, znode, new byte[0]);
1155   }
1156 
1157   /**
1158    * Creates the specified node containing specified data, iff the node does not exist.  Does
1159    * not set a watch and fails silently if the node already exists.
1160    *
1161    * The node created is persistent and open access.
1162    *
1163    * @param zkw zk reference
1164    * @param znode path of node
1165    * @param data a byte array data to store in the znode
1166    * @throws KeeperException if unexpected zookeeper exception
1167    */
1168   public static void createAndFailSilent(ZooKeeperWatcher zkw,
1169       String znode, byte[] data)
1170   throws KeeperException {
1171     createAndFailSilent(zkw,
1172         (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
1173   }
1174 
1175   private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
1176   throws KeeperException {
1177     CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
1178     String znode = create.getPath();
1179     try {
1180       RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
1181       if (zk.exists(znode, false) == null) {
1182         zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
1183       }
1184     } catch(KeeperException.NodeExistsException nee) {
1185     } catch(KeeperException.NoAuthException nee){
1186       try {
1187         if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
1188           // If we failed to create the file and it does not already exist.
1189           throw(nee);
1190         }
1191       } catch (InterruptedException ie) {
1192         zkw.interruptedException(ie);
1193       }
1194 
1195     } catch(InterruptedException ie) {
1196       zkw.interruptedException(ie);
1197     }
1198   }
1199 
1200   /**
1201    * Creates the specified node and all parent nodes required for it to exist.
1202    *
1203    * No watches are set and no errors are thrown if the node already exists.
1204    *
1205    * The nodes created are persistent and open access.
1206    *
1207    * @param zkw zk reference
1208    * @param znode path of node
1209    * @throws KeeperException if unexpected zookeeper exception
1210    */
1211   public static void createWithParents(ZooKeeperWatcher zkw, String znode)
1212   throws KeeperException {
1213     createWithParents(zkw, znode, new byte[0]);
1214   }
1215 
1216   /**
1217    * Creates the specified node and all parent nodes required for it to exist.  The creation of
1218    * parent znodes is not atomic with the leafe znode creation but the data is written atomically
1219    * when the leaf node is created.
1220    *
1221    * No watches are set and no errors are thrown if the node already exists.
1222    *
1223    * The nodes created are persistent and open access.
1224    *
1225    * @param zkw zk reference
1226    * @param znode path of node
1227    * @throws KeeperException if unexpected zookeeper exception
1228    */
1229   public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data)
1230   throws KeeperException {
1231     try {
1232       if(znode == null) {
1233         return;
1234       }
1235       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1236           CreateMode.PERSISTENT);
1237     } catch(KeeperException.NodeExistsException nee) {
1238       return;
1239     } catch(KeeperException.NoNodeException nne) {
1240       createWithParents(zkw, getParent(znode));
1241       createWithParents(zkw, znode, data);
1242     } catch(InterruptedException ie) {
1243       zkw.interruptedException(ie);
1244     }
1245   }
1246 
1247   //
1248   // Deletes
1249   //
1250 
1251   /**
1252    * Delete the specified node.  Sets no watches.  Throws all exceptions.
1253    */
1254   public static void deleteNode(ZooKeeperWatcher zkw, String node)
1255   throws KeeperException {
1256     deleteNode(zkw, node, -1);
1257   }
1258 
1259   /**
1260    * Delete the specified node with the specified version.  Sets no watches.
1261    * Throws all exceptions.
1262    */
1263   public static boolean deleteNode(ZooKeeperWatcher zkw, String node,
1264       int version)
1265   throws KeeperException {
1266     try {
1267       zkw.getRecoverableZooKeeper().delete(node, version);
1268       return true;
1269     } catch(KeeperException.BadVersionException bve) {
1270       return false;
1271     } catch(InterruptedException ie) {
1272       zkw.interruptedException(ie);
1273       return false;
1274     }
1275   }
1276 
1277   /**
1278    * Deletes the specified node.  Fails silent if the node does not exist.
1279    * @param zkw
1280    * @param node
1281    * @throws KeeperException
1282    */
1283   public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
1284   throws KeeperException {
1285     deleteNodeFailSilent(zkw,
1286       (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
1287   }
1288 
1289   private static void deleteNodeFailSilent(ZooKeeperWatcher zkw,
1290       DeleteNodeFailSilent dnfs) throws KeeperException {
1291     DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
1292     try {
1293       zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
1294     } catch(KeeperException.NoNodeException nne) {
1295     } catch(InterruptedException ie) {
1296       zkw.interruptedException(ie);
1297     }
1298   }
1299 
1300 
1301   /**
1302    * Delete the specified node and all of it's children.
1303    * <p>
1304    * If the node does not exist, just returns.
1305    * <p>
1306    * Sets no watches. Throws all exceptions besides dealing with deletion of
1307    * children.
1308    */
1309   public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node)
1310   throws KeeperException {
1311     try {
1312       List<String> children = ZKUtil.listChildrenNoWatch(zkw, node);
1313       // the node is already deleted, so we just finish
1314       if (children == null) return;
1315 
1316       if(!children.isEmpty()) {
1317         for(String child : children) {
1318           deleteNodeRecursively(zkw, joinZNode(node, child));
1319         }
1320       }
1321       //Zookeeper Watches are one time triggers; When children of parent nodes are deleted recursively. 
1322       //Must set another watch, get notified of delete node   
1323       if (zkw.getRecoverableZooKeeper().exists(node, zkw) != null){
1324         zkw.getRecoverableZooKeeper().delete(node, -1);
1325       }
1326     } catch(InterruptedException ie) {
1327       zkw.interruptedException(ie);
1328     }
1329   }
1330 
1331   /**
1332    * Delete all the children of the specified node but not the node itself.
1333    *
1334    * Sets no watches.  Throws all exceptions besides dealing with deletion of
1335    * children.
1336    */
1337   public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
1338   throws KeeperException {
1339     List<String> children = ZKUtil.listChildrenNoWatch(zkw, node);
1340     if (children == null || children.isEmpty()) return;
1341     for(String child : children) {
1342       deleteNodeRecursively(zkw, joinZNode(node, child));
1343     }
1344   }
1345 
1346   /**
1347    * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
1348    * These actions are higher-level than ZKOp actions, which represent
1349    * individual actions in the ZooKeeper API, like create.
1350    */
1351   public abstract static class ZKUtilOp {
1352     private String path;
1353 
1354     private ZKUtilOp(String path) {
1355       this.path = path;
1356     }
1357 
1358     /**
1359      * @return a createAndFailSilent ZKUtilOp
1360      */
1361     public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
1362       return new CreateAndFailSilent(path, data);
1363     }
1364 
1365     /**
1366      * @return a deleteNodeFailSilent ZKUtilOP
1367      */
1368     public static ZKUtilOp deleteNodeFailSilent(String path) {
1369       return new DeleteNodeFailSilent(path);
1370     }
1371 
1372     /**
1373      * @return a setData ZKUtilOp
1374      */
1375     public static ZKUtilOp setData(String path, byte [] data) {
1376       return new SetData(path, data);
1377     }
1378 
1379     /**
1380      * @return path to znode where the ZKOp will occur
1381      */
1382     public String getPath() {
1383       return path;
1384     }
1385 
1386     /**
1387      * ZKUtilOp representing createAndFailSilent in ZooKeeper
1388      * (attempt to create node, ignore error if already exists)
1389      */
1390     public static class CreateAndFailSilent extends ZKUtilOp {
1391       private byte [] data;
1392 
1393       private CreateAndFailSilent(String path, byte [] data) {
1394         super(path);
1395         this.data = data;
1396       }
1397 
1398       public byte[] getData() {
1399         return data;
1400       }
1401 
1402       @Override
1403       public boolean equals(Object o) {
1404         if (this == o) return true;
1405         if (!(o instanceof CreateAndFailSilent)) return false;
1406 
1407         CreateAndFailSilent op = (CreateAndFailSilent) o;
1408         return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
1409       }
1410 
1411       @Override
1412       public int hashCode() {
1413         int ret = 17 + getPath().hashCode() * 31;
1414         return ret * 31 + Bytes.hashCode(data);
1415       }
1416     }
1417 
1418     /**
1419      * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
1420      * (attempt to delete node, ignore error if node doesn't exist)
1421      */
1422     public static class DeleteNodeFailSilent extends ZKUtilOp {
1423       private DeleteNodeFailSilent(String path) {
1424         super(path);
1425       }
1426 
1427       @Override
1428       public boolean equals(Object o) {
1429         if (this == o) return true;
1430         if (!(o instanceof DeleteNodeFailSilent)) return false;
1431 
1432         return super.equals(o);
1433       }
1434 
1435       @Override
1436       public int hashCode() {
1437         return getPath().hashCode();
1438       }
1439     }
1440 
1441     /**
1442      * ZKUtilOp representing setData in ZooKeeper
1443      */
1444     public static class SetData extends ZKUtilOp {
1445       private byte [] data;
1446 
1447       private SetData(String path, byte [] data) {
1448         super(path);
1449         this.data = data;
1450       }
1451 
1452       public byte[] getData() {
1453         return data;
1454       }
1455 
1456       @Override
1457       public boolean equals(Object o) {
1458         if (this == o) return true;
1459         if (!(o instanceof SetData)) return false;
1460 
1461         SetData op = (SetData) o;
1462         return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
1463       }
1464 
1465       @Override
1466       public int hashCode() {
1467         int ret = getPath().hashCode();
1468         return ret * 31 + Bytes.hashCode(data);
1469       }
1470     }
1471   }
1472 
1473   /**
1474    * Convert from ZKUtilOp to ZKOp
1475    */
1476   private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op)
1477   throws UnsupportedOperationException {
1478     if(op == null) return null;
1479 
1480     if (op instanceof CreateAndFailSilent) {
1481       CreateAndFailSilent cafs = (CreateAndFailSilent)op;
1482       return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
1483         CreateMode.PERSISTENT);
1484     } else if (op instanceof DeleteNodeFailSilent) {
1485       DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
1486       return Op.delete(dnfs.getPath(), -1);
1487     } else if (op instanceof SetData) {
1488       SetData sd = (SetData)op;
1489       return Op.setData(sd.getPath(), sd.getData(), -1);
1490     } else {
1491       throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1492         + op.getClass().getName());
1493     }
1494   }
1495 
1496   /**
1497    * If hbase.zookeeper.useMulti is true, use ZooKeeper's multi-update functionality.
1498    * Otherwise, run the list of operations sequentially.
1499    *
1500    * If all of the following are true:
1501    * - runSequentialOnMultiFailure is true
1502    * - hbase.zookeeper.useMulti is true
1503    * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
1504    * Then:
1505    * - we retry the operations one-by-one (sequentially)
1506    *
1507    * Note *: an example is receiving a NodeExistsException from a "create" call.  Without multi,
1508    * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
1509    * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
1510    * This will cause all operations in the multi to fail, however, because
1511    * the NodeExistsException that zk.create throws will fail the multi transaction.
1512    * In this case, if the previous conditions hold, the commands are run sequentially, which should
1513    * result in the correct final state, but means that the operations will not run atomically.
1514    *
1515    * @throws KeeperException
1516    */
1517   public static void multiOrSequential(ZooKeeperWatcher zkw, List<ZKUtilOp> ops,
1518       boolean runSequentialOnMultiFailure) throws KeeperException {
1519     if (ops == null) return;
1520     boolean useMulti = zkw.getConfiguration().getBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
1521 
1522     if (useMulti) {
1523       List<Op> zkOps = new LinkedList<Op>();
1524       for (ZKUtilOp op : ops) {
1525         zkOps.add(toZooKeeperOp(zkw, op));
1526       }
1527       try {
1528         zkw.getRecoverableZooKeeper().multi(zkOps);
1529       } catch (KeeperException ke) {
1530        switch (ke.code()) {
1531          case NODEEXISTS:
1532          case NONODE:
1533          case BADVERSION:
1534          case NOAUTH:
1535            // if we get an exception that could be solved by running sequentially
1536            // (and the client asked us to), then break out and run sequentially
1537            if (runSequentialOnMultiFailure) {
1538              LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
1539                + "  Attempting to run operations sequentially because"
1540                + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
1541              processSequentially(zkw, ops);
1542              break;
1543            }
1544           default:
1545             throw ke;
1546         }
1547       } catch (InterruptedException ie) {
1548         zkw.interruptedException(ie);
1549       }
1550     } else {
1551       // run sequentially
1552       processSequentially(zkw, ops);
1553     }
1554 
1555   }
1556 
1557   private static void processSequentially(ZooKeeperWatcher zkw, List<ZKUtilOp> ops)
1558       throws KeeperException, NoNodeException {
1559     for (ZKUtilOp op : ops) {
1560       if (op instanceof CreateAndFailSilent) {
1561         createAndFailSilent(zkw, (CreateAndFailSilent) op);
1562       } else if (op instanceof DeleteNodeFailSilent) {
1563         deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
1564       } else if (op instanceof SetData) {
1565         setData(zkw, (SetData) op);
1566       } else {
1567         throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1568             + op.getClass().getName());
1569       }
1570     }
1571   }
1572 
1573   //
1574   // ZooKeeper cluster information
1575   //
1576 
1577   /** @return String dump of everything in ZooKeeper. */
1578   public static String dump(ZooKeeperWatcher zkw) {
1579     StringBuilder sb = new StringBuilder();
1580     try {
1581       sb.append("HBase is rooted at ").append(zkw.baseZNode);
1582       sb.append("\nActive master address: ");
1583       try {
1584         sb.append(MasterAddressTracker.getMasterAddress(zkw));
1585       } catch (IOException e) {
1586         sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
1587       }
1588       sb.append("\nBackup master addresses:");
1589       for (String child : listChildrenNoWatch(zkw,
1590                                               zkw.backupMasterAddressesZNode)) {
1591         sb.append("\n ").append(child);
1592       }
1593       sb.append("\nRegion server holding hbase:meta: "
1594         + new MetaTableLocator().getMetaRegionLocation(zkw));
1595       sb.append("\nRegion servers:");
1596       for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
1597         sb.append("\n ").append(child);
1598       }
1599       try {
1600         getReplicationZnodesDump(zkw, sb);
1601       } catch (KeeperException ke) {
1602         LOG.warn("Couldn't get the replication znode dump", ke);
1603       }
1604       sb.append("\nQuorum Server Statistics:");
1605       String[] servers = zkw.getQuorum().split(",");
1606       for (String server : servers) {
1607         sb.append("\n ").append(server);
1608         try {
1609           String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
1610 
1611           if (stat == null) {
1612             sb.append("[Error] invalid quorum server: " + server);
1613             break;
1614           }
1615 
1616           for (String s : stat) {
1617             sb.append("\n  ").append(s);
1618           }
1619         } catch (Exception e) {
1620           sb.append("\n  ERROR: ").append(e.getMessage());
1621         }
1622       }
1623     } catch (KeeperException ke) {
1624       sb.append("\nFATAL ZooKeeper Exception!\n");
1625       sb.append("\n" + ke.getMessage());
1626     }
1627     return sb.toString();
1628   }
1629 
1630   /**
1631    * Appends replication znodes to the passed StringBuilder.
1632    * @param zkw
1633    * @param sb
1634    * @throws KeeperException
1635    */
1636   private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb)
1637       throws KeeperException {
1638     String replicationZNodeName = zkw.getConfiguration().get("zookeeper.znode.replication",
1639       "replication");
1640     String replicationZnode = joinZNode(zkw.baseZNode, replicationZNodeName);
1641     if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
1642     // do a ls -r on this znode
1643     sb.append("\n").append(replicationZnode).append(": ");
1644     List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
1645     for (String child : children) {
1646       String znode = joinZNode(replicationZnode, child);
1647       if (child.equals(zkw.getConfiguration().get("zookeeper.znode.replication.peers", "peers"))) {
1648         appendPeersZnodes(zkw, znode, sb);
1649       } else if (child.equals(zkw.getConfiguration().
1650           get("zookeeper.znode.replication.rs", "rs"))) {
1651         appendRSZnodes(zkw, znode, sb);
1652       }
1653     }
1654   }
1655 
1656   private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
1657       throws KeeperException {
1658     List<String> stack = new LinkedList<String>();
1659     stack.add(znode);
1660     do {
1661       String znodeToProcess = stack.remove(stack.size() - 1);
1662       sb.append("\n").append(znodeToProcess).append(": ");
1663       byte[] data;
1664       try {
1665         data = ZKUtil.getData(zkw, znodeToProcess);
1666       } catch (InterruptedException e) {
1667         zkw.interruptedException(e);
1668         return;
1669       }
1670       if (data != null && data.length > 0) { // log position
1671         long position = 0;
1672         try {
1673           position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
1674           sb.append(position);
1675         } catch (DeserializationException ignored) {
1676         } catch (InterruptedException e) {
1677           zkw.interruptedException(e);
1678           return;
1679         }
1680       }
1681       for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
1682         stack.add(ZKUtil.joinZNode(znodeToProcess, zNodeChild));
1683       }
1684     } while (stack.size() > 0);
1685   }
1686 
1687   private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode,
1688     StringBuilder sb) throws KeeperException {
1689     int pblen = ProtobufUtil.lengthOfPBMagic();
1690     sb.append("\n").append(peersZnode).append(": ");
1691     for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
1692       String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode);
1693       byte[] data;
1694       try {
1695         data = ZKUtil.getData(zkw, znodeToProcess);
1696       } catch (InterruptedException e) {
1697         zkw.interruptedException(e);
1698         return;
1699       }
1700       // parse the data of the above peer znode.
1701       try {
1702       String clusterKey = ZooKeeperProtos.ReplicationPeer.newBuilder().
1703         mergeFrom(data, pblen, data.length - pblen).getClusterkey();
1704       sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
1705       // add the peer-state.
1706       appendPeerState(zkw, znodeToProcess, sb);
1707       } catch (InvalidProtocolBufferException ipbe) {
1708         LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
1709       }
1710     }
1711   }
1712 
1713   private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess,
1714       StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
1715     String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
1716       "peer-state");
1717     int pblen = ProtobufUtil.lengthOfPBMagic();
1718     for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
1719       if (!child.equals(peerState)) continue;
1720       String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child);
1721       sb.append("\n").append(peerStateZnode).append(": ");
1722       byte[] peerStateData;
1723       try {
1724         peerStateData = ZKUtil.getData(zkw, peerStateZnode);
1725         sb.append(ZooKeeperProtos.ReplicationState.newBuilder()
1726             .mergeFrom(peerStateData, pblen, peerStateData.length - pblen).getState().name());
1727       } catch (InterruptedException e) {
1728         zkw.interruptedException(e);
1729         return;
1730       }
1731     }
1732   }
1733 
1734   /**
1735    * Gets the statistics from the given server.
1736    *
1737    * @param server  The server to get the statistics from.
1738    * @param timeout  The socket timeout to use.
1739    * @return The array of response strings.
1740    * @throws IOException When the socket communication fails.
1741    */
1742   public static String[] getServerStats(String server, int timeout)
1743   throws IOException {
1744     String[] sp = server.split(":");
1745     if (sp == null || sp.length == 0) {
1746       return null;
1747     }
1748 
1749     String host = sp[0];
1750     int port = sp.length > 1 ? Integer.parseInt(sp[1])
1751         : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
1752 
1753     Socket socket = new Socket();
1754     InetSocketAddress sockAddr = new InetSocketAddress(host, port);
1755     socket.connect(sockAddr, timeout);
1756 
1757     socket.setSoTimeout(timeout);
1758     PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
1759     BufferedReader in = new BufferedReader(new InputStreamReader(
1760       socket.getInputStream()));
1761     out.println("stat");
1762     out.flush();
1763     ArrayList<String> res = new ArrayList<String>();
1764     while (true) {
1765       String line = in.readLine();
1766       if (line != null) {
1767         res.add(line);
1768       } else {
1769         break;
1770       }
1771     }
1772     socket.close();
1773     return res.toArray(new String[res.size()]);
1774   }
1775 
1776   private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
1777       final String znode, final byte [] data, final boolean watcherSet) {
1778     if (!LOG.isTraceEnabled()) return;
1779     LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
1780       " byte(s) of data from znode " + znode +
1781       (watcherSet? " and set watcher; ": "; data=") +
1782       (data == null? "null": data.length == 0? "empty": (
1783           znode.startsWith(zkw.metaServerZNode)?
1784             getServerNameOrEmptyString(data):
1785           znode.startsWith(zkw.backupMasterAddressesZNode)?
1786             getServerNameOrEmptyString(data):
1787           StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
1788   }
1789 
1790   private static String getServerNameOrEmptyString(final byte [] data) {
1791     try {
1792       return ServerName.parseFrom(data).toString();
1793     } catch (DeserializationException e) {
1794       return "";
1795     }
1796   }
1797 
1798   /**
1799    * Waits for HBase installation's base (parent) znode to become available.
1800    * @throws IOException on ZK errors
1801    */
1802   public static void waitForBaseZNode(Configuration conf) throws IOException {
1803     LOG.info("Waiting until the base znode is available");
1804     String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
1805         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
1806     ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
1807         conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
1808         HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
1809 
1810     final int maxTimeMs = 10000;
1811     final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
1812 
1813     KeeperException keeperEx = null;
1814     try {
1815       try {
1816         for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
1817           try {
1818             if (zk.exists(parentZNode, false) != null) {
1819               LOG.info("Parent znode exists: " + parentZNode);
1820               keeperEx = null;
1821               break;
1822             }
1823           } catch (KeeperException e) {
1824             keeperEx = e;
1825           }
1826           Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
1827         }
1828       } finally {
1829         zk.close();
1830       }
1831     } catch (InterruptedException ex) {
1832       Thread.currentThread().interrupt();
1833     }
1834 
1835     if (keeperEx != null) {
1836       throw new IOException(keeperEx);
1837     }
1838   }
1839 
1840   /**
1841    * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
1842    * Used when can't let a {@link DeserializationException} out w/o changing public API.
1843    * @param e Exception to convert
1844    * @return Converted exception
1845    */
1846   public static KeeperException convert(final DeserializationException e) {
1847     KeeperException ke = new KeeperException.DataInconsistencyException();
1848     ke.initCause(e);
1849     return ke;
1850   }
1851 
1852   /**
1853    * Recursively print the current state of ZK (non-transactional)
1854    * @param root name of the root directory in zk to print
1855    * @throws KeeperException
1856    */
1857   public static void logZKTree(ZooKeeperWatcher zkw, String root) {
1858     if (!LOG.isDebugEnabled()) return;
1859     LOG.debug("Current zk system:");
1860     String prefix = "|-";
1861     LOG.debug(prefix + root);
1862     try {
1863       logZKTree(zkw, root, prefix);
1864     } catch (KeeperException e) {
1865       throw new RuntimeException(e);
1866     }
1867   }
1868 
1869   /**
1870    * Helper method to print the current state of the ZK tree.
1871    * @see #logZKTree(ZooKeeperWatcher, String)
1872    * @throws KeeperException if an unexpected exception occurs
1873    */
1874   protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix) throws KeeperException {
1875     List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
1876     if (children == null) return;
1877     for (String child : children) {
1878       LOG.debug(prefix + child);
1879       String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
1880       logZKTree(zkw, node, prefix + "---");
1881     }
1882   }
1883 
1884   /**
1885    * @param position
1886    * @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
1887    *         for use as content of an wal position in a replication queue.
1888    */
1889   public static byte[] positionToByteArray(final long position) {
1890     byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
1891         .build().toByteArray();
1892     return ProtobufUtil.prependPBMagic(bytes);
1893   }
1894 
1895   /**
1896    * @param bytes - Content of a WAL position znode.
1897    * @return long - The current WAL position.
1898    * @throws DeserializationException
1899    */
1900   public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
1901     if (bytes == null) {
1902       throw new DeserializationException("Unable to parse null WAL position.");
1903     }
1904     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
1905       int pblen = ProtobufUtil.lengthOfPBMagic();
1906       ZooKeeperProtos.ReplicationHLogPosition.Builder builder =
1907           ZooKeeperProtos.ReplicationHLogPosition.newBuilder();
1908       ZooKeeperProtos.ReplicationHLogPosition position;
1909       try {
1910         position = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
1911       } catch (InvalidProtocolBufferException e) {
1912         throw new DeserializationException(e);
1913       }
1914       return position.getPosition();
1915     } else {
1916       if (bytes.length > 0) {
1917         return Bytes.toLong(bytes);
1918       }
1919       return 0;
1920     }
1921   }
1922 
1923   /**
1924    * @param regionLastFlushedSequenceId the flushed sequence id of a region which is the min of its
1925    *          store max seq ids
1926    * @param storeSequenceIds column family to sequence Id map
1927    * @return Serialized protobuf of <code>RegionSequenceIds</code> with pb magic prefix prepended
1928    *         suitable for use to filter wal edits in distributedLogReplay mode
1929    */
1930   public static byte[] regionSequenceIdsToByteArray(final Long regionLastFlushedSequenceId,
1931       final Map<byte[], Long> storeSequenceIds) {
1932     ZooKeeperProtos.RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
1933         ZooKeeperProtos.RegionStoreSequenceIds.newBuilder();
1934     ZooKeeperProtos.StoreSequenceId.Builder storeSequenceIdBuilder =
1935         ZooKeeperProtos.StoreSequenceId.newBuilder();
1936     if (storeSequenceIds != null) {
1937       for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
1938         byte[] columnFamilyName = e.getKey();
1939         Long curSeqId = e.getValue();
1940         storeSequenceIdBuilder.setFamilyName(ByteStringer.wrap(columnFamilyName));
1941         storeSequenceIdBuilder.setSequenceId(curSeqId);
1942         regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
1943         storeSequenceIdBuilder.clear();
1944       }
1945     }
1946     regionSequenceIdsBuilder.setLastFlushedSequenceId(regionLastFlushedSequenceId);
1947     byte[] result = regionSequenceIdsBuilder.build().toByteArray();
1948     return ProtobufUtil.prependPBMagic(result);
1949   }
1950 
1951   /**
1952    * @param bytes Content of serialized data of RegionStoreSequenceIds
1953    * @return a RegionStoreSequenceIds object
1954    * @throws DeserializationException
1955    */
1956   public static RegionStoreSequenceIds parseRegionStoreSequenceIds(final byte[] bytes)
1957       throws DeserializationException {
1958     if (bytes == null || !ProtobufUtil.isPBMagicPrefix(bytes)) {
1959       throw new DeserializationException("Unable to parse RegionStoreSequenceIds.");
1960     }
1961     RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
1962         ZooKeeperProtos.RegionStoreSequenceIds.newBuilder();
1963     int pblen = ProtobufUtil.lengthOfPBMagic();
1964     RegionStoreSequenceIds storeIds = null;
1965     try {
1966       storeIds = regionSequenceIdsBuilder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
1967     } catch (InvalidProtocolBufferException e) {
1968       throw new DeserializationException(e);
1969     }
1970     return storeIds;
1971   }
1972 }