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