1   
2   
3   
4   
5   
6   
7   
8   
9   
10  
11  
12  
13  
14  
15  
16  
17  
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.Deque;
30  import java.util.HashMap;
31  import java.util.LinkedList;
32  import java.util.List;
33  import java.util.Map;
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.AuthUtil;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.ServerName;
46  import org.apache.hadoop.hbase.classification.InterfaceAudience;
47  import org.apache.hadoop.hbase.exceptions.DeserializationException;
48  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
49  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
50  import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
51  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
52  import org.apache.hadoop.hbase.security.Superusers;
53  import org.apache.hadoop.hbase.util.ByteStringer;
54  import org.apache.hadoop.hbase.util.Bytes;
55  import org.apache.hadoop.hbase.util.Threads;
56  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.CreateAndFailSilent;
57  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent;
58  import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData;
59  import org.apache.hadoop.security.SecurityUtil;
60  import org.apache.hadoop.security.authentication.util.KerberosUtil;
61  import org.apache.zookeeper.AsyncCallback;
62  import org.apache.zookeeper.CreateMode;
63  import org.apache.zookeeper.KeeperException;
64  import org.apache.zookeeper.KeeperException.NoNodeException;
65  import org.apache.zookeeper.Op;
66  import org.apache.zookeeper.Watcher;
67  import org.apache.zookeeper.ZooDefs.Ids;
68  import org.apache.zookeeper.ZooDefs.Perms;
69  import org.apache.zookeeper.ZooKeeper;
70  import org.apache.zookeeper.client.ZooKeeperSaslClient;
71  import org.apache.zookeeper.data.ACL;
72  import org.apache.zookeeper.data.Id;
73  import org.apache.zookeeper.data.Stat;
74  import org.apache.zookeeper.proto.CreateRequest;
75  import org.apache.zookeeper.proto.DeleteRequest;
76  import org.apache.zookeeper.proto.SetDataRequest;
77  import org.apache.zookeeper.server.ZooKeeperSaslServer;
78  
79  import com.google.protobuf.InvalidProtocolBufferException;
80  
81  
82  
83  
84  
85  
86  
87  
88  
89  
90  @InterfaceAudience.Private
91  public class ZKUtil {
92    private static final Log LOG = LogFactory.getLog(ZKUtil.class);
93  
94    
95    public static final char ZNODE_PATH_SEPARATOR = '/';
96    private static int zkDumpConnectionTimeOut;
97  
98    
99  
100 
101 
102 
103 
104 
105 
106 
107 
108 
109   public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher)
110   throws IOException {
111     String ensemble = ZKConfig.getZKQuorumServersString(conf);
112     return connect(conf, ensemble, watcher);
113   }
114 
115   public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
116       Watcher watcher)
117   throws IOException {
118     return connect(conf, ensemble, watcher, null);
119   }
120 
121   public static RecoverableZooKeeper connect(Configuration conf, String ensemble,
122       Watcher watcher, final String identifier)
123   throws IOException {
124     if(ensemble == null) {
125       throw new IOException("Unable to determine ZooKeeper ensemble");
126     }
127     int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
128         HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
129     if (LOG.isTraceEnabled()) {
130       LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble);
131     }
132     int retry = conf.getInt("zookeeper.recovery.retry", 3);
133     int retryIntervalMillis =
134       conf.getInt("zookeeper.recovery.retry.intervalmill", 1000);
135     zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout",
136         1000);
137     return new RecoverableZooKeeper(ensemble, timeout, watcher,
138         retry, retryIntervalMillis, identifier);
139   }
140 
141   
142 
143 
144 
145 
146 
147 
148 
149 
150 
151 
152 
153 
154 
155   public static void loginServer(Configuration conf, String keytabFileKey,
156       String userNameKey, String hostname) throws IOException {
157     login(conf, keytabFileKey, userNameKey, hostname,
158           ZooKeeperSaslServer.LOGIN_CONTEXT_NAME_KEY,
159           JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME);
160   }
161 
162   
163 
164 
165 
166 
167 
168 
169 
170 
171 
172 
173 
174 
175 
176   public static void loginClient(Configuration conf, String keytabFileKey,
177       String userNameKey, String hostname) throws IOException {
178     login(conf, keytabFileKey, userNameKey, hostname,
179           ZooKeeperSaslClient.LOGIN_CONTEXT_NAME_KEY,
180           JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME);
181   }
182 
183   
184 
185 
186 
187 
188 
189 
190 
191 
192 
193 
194 
195 
196 
197 
198 
199   private static void login(Configuration conf, String keytabFileKey,
200       String userNameKey, String hostname,
201       String loginContextProperty, String loginContextName)
202       throws IOException {
203     if (!isSecureZooKeeper(conf))
204       return;
205 
206     
207     
208     if (System.getProperty("java.security.auth.login.config") != null)
209       return;
210 
211     
212     String keytabFilename = conf.get(keytabFileKey);
213     if (keytabFilename == null) {
214       LOG.warn("no keytab specified for: " + keytabFileKey);
215       return;
216     }
217 
218     String principalConfig = conf.get(userNameKey, System.getProperty("user.name"));
219     String principalName = SecurityUtil.getServerPrincipal(principalConfig, hostname);
220 
221     
222     
223     
224     JaasConfiguration jaasConf = new JaasConfiguration(loginContextName,
225         principalName, keytabFilename);
226     javax.security.auth.login.Configuration.setConfiguration(jaasConf);
227     System.setProperty(loginContextProperty, loginContextName);
228   }
229 
230   
231 
232 
233   private static class JaasConfiguration extends javax.security.auth.login.Configuration {
234     private static final String SERVER_KEYTAB_KERBEROS_CONFIG_NAME =
235       "zookeeper-server-keytab-kerberos";
236     private static final String CLIENT_KEYTAB_KERBEROS_CONFIG_NAME =
237       "zookeeper-client-keytab-kerberos";
238 
239     private static final Map<String, String> BASIC_JAAS_OPTIONS =
240       new HashMap<String,String>();
241     static {
242       String jaasEnvVar = System.getenv("HBASE_JAAS_DEBUG");
243       if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) {
244         BASIC_JAAS_OPTIONS.put("debug", "true");
245       }
246     }
247 
248     private static final Map<String,String> KEYTAB_KERBEROS_OPTIONS =
249       new HashMap<String,String>();
250     static {
251       KEYTAB_KERBEROS_OPTIONS.put("doNotPrompt", "true");
252       KEYTAB_KERBEROS_OPTIONS.put("storeKey", "true");
253       KEYTAB_KERBEROS_OPTIONS.put("refreshKrb5Config", "true");
254       KEYTAB_KERBEROS_OPTIONS.putAll(BASIC_JAAS_OPTIONS);
255     }
256 
257     private static final AppConfigurationEntry KEYTAB_KERBEROS_LOGIN =
258       new AppConfigurationEntry(KerberosUtil.getKrb5LoginModuleName(),
259                                 LoginModuleControlFlag.REQUIRED,
260                                 KEYTAB_KERBEROS_OPTIONS);
261 
262     private static final AppConfigurationEntry[] KEYTAB_KERBEROS_CONF =
263       new AppConfigurationEntry[]{KEYTAB_KERBEROS_LOGIN};
264 
265     private javax.security.auth.login.Configuration baseConfig;
266     private final String loginContextName;
267     private final boolean useTicketCache;
268     private final String keytabFile;
269     private final String principal;
270 
271     public JaasConfiguration(String loginContextName, String principal) {
272       this(loginContextName, principal, null, true);
273     }
274 
275     public JaasConfiguration(String loginContextName, String principal, String keytabFile) {
276       this(loginContextName, principal, keytabFile, keytabFile == null || keytabFile.length() == 0);
277     }
278 
279     private JaasConfiguration(String loginContextName, String principal,
280                              String keytabFile, boolean useTicketCache) {
281       try {
282         this.baseConfig = javax.security.auth.login.Configuration.getConfiguration();
283       } catch (SecurityException e) {
284         this.baseConfig = null;
285       }
286       this.loginContextName = loginContextName;
287       this.useTicketCache = useTicketCache;
288       this.keytabFile = keytabFile;
289       this.principal = principal;
290       LOG.info("JaasConfiguration loginContextName=" + loginContextName +
291                " principal=" + principal + " useTicketCache=" + useTicketCache +
292                " keytabFile=" + keytabFile);
293     }
294 
295     @Override
296     public AppConfigurationEntry[] getAppConfigurationEntry(String appName) {
297       if (loginContextName.equals(appName)) {
298         if (!useTicketCache) {
299           KEYTAB_KERBEROS_OPTIONS.put("keyTab", keytabFile);
300           KEYTAB_KERBEROS_OPTIONS.put("useKeyTab", "true");
301         }
302         KEYTAB_KERBEROS_OPTIONS.put("principal", principal);
303         KEYTAB_KERBEROS_OPTIONS.put("useTicketCache", useTicketCache ? "true" : "false");
304         return KEYTAB_KERBEROS_CONF;
305       }
306       if (baseConfig != null) return baseConfig.getAppConfigurationEntry(appName);
307       return(null);
308     }
309   }
310 
311   
312   
313   
314 
315   
316 
317 
318 
319 
320 
321 
322 
323 
324 
325   public static String joinZNode(String prefix, String suffix) {
326     return prefix + ZNODE_PATH_SEPARATOR + suffix;
327   }
328 
329   
330 
331 
332 
333 
334   public static String getParent(String node) {
335     int idx = node.lastIndexOf(ZNODE_PATH_SEPARATOR);
336     return idx <= 0 ? null : node.substring(0, idx);
337   }
338 
339   
340 
341 
342 
343 
344   public static String getNodeName(String path) {
345     return path.substring(path.lastIndexOf("/")+1);
346   }
347 
348   
349   
350   
351 
352   
353 
354 
355 
356 
357 
358 
359 
360 
361 
362   public static boolean watchAndCheckExists(ZooKeeperWatcher zkw, String znode)
363   throws KeeperException {
364     try {
365       Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw);
366       boolean exists = s != null ? true : false;
367       if (exists) {
368         LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode));
369       } else {
370         LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode));
371       }
372       return exists;
373     } catch (KeeperException e) {
374       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
375       zkw.keeperException(e);
376       return false;
377     } catch (InterruptedException e) {
378       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
379       zkw.interruptedException(e);
380       return false;
381     }
382   }
383 
384   
385 
386 
387 
388 
389 
390 
391 
392 
393 
394   public static boolean setWatchIfNodeExists(ZooKeeperWatcher zkw, String znode)
395       throws KeeperException {
396     try {
397       zkw.getRecoverableZooKeeper().getData(znode, true, null);
398       return true;
399     } catch (NoNodeException e) {
400       return false;
401     } catch (InterruptedException e) {
402       LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e);
403       zkw.interruptedException(e);
404       return false;
405     }
406   }
407 
408   
409 
410 
411 
412 
413 
414 
415 
416   public static int checkExists(ZooKeeperWatcher zkw, String znode)
417   throws KeeperException {
418     try {
419       Stat s = zkw.getRecoverableZooKeeper().exists(znode, null);
420       return s != null ? s.getVersion() : -1;
421     } catch (KeeperException e) {
422       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
423       zkw.keeperException(e);
424       return -1;
425     } catch (InterruptedException e) {
426       LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e);
427       zkw.interruptedException(e);
428       return -1;
429     }
430   }
431 
432   
433   
434   
435 
436   
437 
438 
439 
440 
441 
442 
443 
444 
445 
446 
447 
448 
449 
450 
451 
452   public static List<String> listChildrenAndWatchForNewChildren(
453       ZooKeeperWatcher zkw, String znode)
454   throws KeeperException {
455     try {
456       List<String> children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw);
457       return children;
458     } catch(KeeperException.NoNodeException ke) {
459       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
460           "because node does not exist (not an error)"));
461       return null;
462     } catch (KeeperException e) {
463       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
464       zkw.keeperException(e);
465       return null;
466     } catch (InterruptedException e) {
467       LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e);
468       zkw.interruptedException(e);
469       return null;
470     }
471   }
472 
473   
474 
475 
476 
477 
478 
479 
480 
481 
482   public static List<String> listChildrenAndWatchThem(ZooKeeperWatcher zkw,
483       String znode) throws KeeperException {
484     List<String> children = listChildrenAndWatchForNewChildren(zkw, znode);
485     if (children == null) {
486       return null;
487     }
488     for (String child : children) {
489       watchAndCheckExists(zkw, joinZNode(znode, child));
490     }
491     return children;
492   }
493 
494   
495 
496 
497 
498 
499 
500 
501 
502 
503 
504 
505 
506 
507 
508   public static List<String> listChildrenNoWatch(ZooKeeperWatcher zkw, String znode)
509   throws KeeperException {
510     List<String> children = null;
511     try {
512       
513       children = zkw.getRecoverableZooKeeper().getChildren(znode, null);
514     } catch(KeeperException.NoNodeException nne) {
515       return null;
516     } catch(InterruptedException ie) {
517       zkw.interruptedException(ie);
518     }
519     return children;
520   }
521 
522   
523 
524 
525 
526   @Deprecated
527   public static class NodeAndData {
528     private String node;
529     private byte [] data;
530     public NodeAndData(String node, byte [] data) {
531       this.node = node;
532       this.data = data;
533     }
534     public String getNode() {
535       return node;
536     }
537     public byte [] getData() {
538       return data;
539     }
540     @Override
541     public String toString() {
542       return node;
543     }
544     public boolean isEmpty() {
545       return (data == null || data.length == 0);
546     }
547   }
548 
549   
550 
551 
552 
553 
554 
555 
556 
557 
558 
559 
560 
561 
562 
563 
564 
565   public static boolean nodeHasChildren(ZooKeeperWatcher zkw, String znode)
566   throws KeeperException {
567     try {
568       return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty();
569     } catch(KeeperException.NoNodeException ke) {
570       LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " +
571       "because node does not exist (not an error)"));
572       return false;
573     } catch (KeeperException e) {
574       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
575       zkw.keeperException(e);
576       return false;
577     } catch (InterruptedException e) {
578       LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e);
579       zkw.interruptedException(e);
580       return false;
581     }
582   }
583 
584   
585 
586 
587 
588 
589 
590 
591 
592 
593 
594 
595 
596 
597   public static int getNumberOfChildren(ZooKeeperWatcher zkw, String znode)
598   throws KeeperException {
599     try {
600       Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null);
601       return stat == null ? 0 : stat.getNumChildren();
602     } catch(KeeperException e) {
603       LOG.warn(zkw.prefix("Unable to get children of node " + znode));
604       zkw.keeperException(e);
605     } catch(InterruptedException e) {
606       zkw.interruptedException(e);
607     }
608     return 0;
609   }
610 
611   
612   
613   
614 
615   
616 
617 
618 
619 
620   public static byte [] getData(ZooKeeperWatcher zkw, String znode)
621       throws KeeperException, InterruptedException {
622     try {
623       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, null);
624       logRetrievedMsg(zkw, znode, data, false);
625       return data;
626     } catch (KeeperException.NoNodeException e) {
627       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
628           "because node does not exist (not an error)"));
629       return null;
630     } catch (KeeperException e) {
631       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
632       zkw.keeperException(e);
633       return null;
634     }
635   }
636 
637   
638 
639 
640 
641 
642 
643 
644 
645 
646 
647 
648   public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
649   throws KeeperException {
650     return getDataInternal(zkw, znode, null, true);
651   }
652 
653   
654 
655 
656 
657 
658 
659 
660 
661 
662 
663 
664 
665   public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
666       Stat stat) throws KeeperException {
667     return getDataInternal(zkw, znode, stat, true);
668   }
669 
670   private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
671       boolean watcherSet)
672       throws KeeperException {
673     try {
674       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
675       logRetrievedMsg(zkw, znode, data, watcherSet);
676       return data;
677     } catch (KeeperException.NoNodeException e) {
678       
679       
680       LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " +
681         "because node does not exist (not an error)"));
682       return null;
683     } catch (KeeperException e) {
684       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
685       zkw.keeperException(e);
686       return null;
687     } catch (InterruptedException e) {
688       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
689       zkw.interruptedException(e);
690       return null;
691     }
692   }
693 
694   
695 
696 
697 
698 
699 
700 
701 
702 
703 
704 
705 
706 
707 
708 
709   public static byte [] getDataNoWatch(ZooKeeperWatcher zkw, String znode,
710       Stat stat)
711   throws KeeperException {
712     try {
713       byte [] data = zkw.getRecoverableZooKeeper().getData(znode, null, stat);
714       logRetrievedMsg(zkw, znode, data, false);
715       return data;
716     } catch (KeeperException.NoNodeException e) {
717       LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " +
718           "because node does not exist (not necessarily an error)"));
719       return null;
720     } catch (KeeperException e) {
721       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
722       zkw.keeperException(e);
723       return null;
724     } catch (InterruptedException e) {
725       LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
726       zkw.interruptedException(e);
727       return null;
728     }
729   }
730 
731   
732 
733 
734 
735 
736 
737 
738 
739 
740 
741 
742 
743 
744 
745 
746 
747 
748   public static List<NodeAndData> getChildDataAndWatchForNewChildren(
749       ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
750     List<String> nodes =
751       ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
752     if (nodes != null) {
753       List<NodeAndData> newNodes = new ArrayList<NodeAndData>();
754       for (String node : nodes) {
755         String nodePath = ZKUtil.joinZNode(baseNode, node);
756         byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
757         newNodes.add(new NodeAndData(nodePath, data));
758       }
759       return newNodes;
760     }
761     return null;
762   }
763 
764   
765 
766 
767 
768 
769 
770 
771 
772 
773 
774 
775 
776 
777 
778 
779 
780   public static void updateExistingNodeData(ZooKeeperWatcher zkw, String znode,
781       byte [] data, int expectedVersion)
782   throws KeeperException {
783     try {
784       zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion);
785     } catch(InterruptedException ie) {
786       zkw.interruptedException(ie);
787     }
788   }
789 
790   
791   
792   
793 
794   
795 
796 
797 
798 
799 
800 
801 
802 
803 
804 
805 
806 
807 
808 
809 
810 
811 
812 
813 
814   public static boolean setData(ZooKeeperWatcher zkw, String znode,
815       byte [] data, int expectedVersion)
816   throws KeeperException, KeeperException.NoNodeException {
817     try {
818       return zkw.getRecoverableZooKeeper().setData(znode, data, expectedVersion) != null;
819     } catch (InterruptedException e) {
820       zkw.interruptedException(e);
821       return false;
822     }
823   }
824 
825   
826 
827 
828 
829 
830 
831 
832 
833 
834   public static void createSetData(final ZooKeeperWatcher zkw, final String znode,
835       final byte [] data)
836   throws KeeperException {
837     if (checkExists(zkw, znode) == -1) {
838       ZKUtil.createWithParents(zkw, znode, data);
839     } else {
840       ZKUtil.setData(zkw, znode, data);
841     }
842   }
843 
844   
845 
846 
847 
848 
849 
850 
851 
852 
853 
854 
855 
856 
857 
858 
859 
860   public static void setData(ZooKeeperWatcher zkw, String znode, byte [] data)
861   throws KeeperException, KeeperException.NoNodeException {
862     setData(zkw, (SetData)ZKUtilOp.setData(znode, data));
863   }
864 
865   private static void setData(ZooKeeperWatcher zkw, SetData setData)
866   throws KeeperException, KeeperException.NoNodeException {
867     SetDataRequest sd = (SetDataRequest)toZooKeeperOp(zkw, setData).toRequestRecord();
868     setData(zkw, sd.getPath(), sd.getData(), sd.getVersion());
869   }
870 
871   
872 
873 
874 
875 
876   public static boolean isSecureZooKeeper(Configuration conf) {
877     
878     
879     try {
880       javax.security.auth.login.Configuration testConfig =
881           javax.security.auth.login.Configuration.getConfiguration();
882       if (testConfig.getAppConfigurationEntry("Client") == null
883           && testConfig.getAppConfigurationEntry(
884             JaasConfiguration.CLIENT_KEYTAB_KERBEROS_CONFIG_NAME) == null
885           && testConfig.getAppConfigurationEntry(
886               JaasConfiguration.SERVER_KEYTAB_KERBEROS_CONFIG_NAME) == null
887           && conf.get(HConstants.ZK_CLIENT_KERBEROS_PRINCIPAL) == null
888           && conf.get(HConstants.ZK_SERVER_KERBEROS_PRINCIPAL) == null) {
889               
890         return false;
891       }
892     } catch(Exception e) {
893       
894       return false;
895     }
896 
897     
898     return "kerberos".equalsIgnoreCase(conf.get("hbase.security.authentication"));
899   }
900 
901   private static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node) {
902     return createACL(zkw, node, isSecureZooKeeper(zkw.getConfiguration()));
903   }
904 
905   public static ArrayList<ACL> createACL(ZooKeeperWatcher zkw, String node,
906     boolean isSecureZooKeeper) {
907     if (!node.startsWith(zkw.baseZNode)) {
908       return Ids.OPEN_ACL_UNSAFE;
909     }
910     if (isSecureZooKeeper) {
911       ArrayList<ACL> acls = new ArrayList<ACL>();
912       
913       String[] superUsers = zkw.getConfiguration().getStrings(Superusers.SUPERUSER_CONF_KEY);
914       if (superUsers != null) {
915         List<String> groups = new ArrayList<String>();
916         for (String user : superUsers) {
917           if (user.startsWith(AuthUtil.GROUP_PREFIX)) {
918             
919             groups.add(user);
920           } else {
921             acls.add(new ACL(Perms.ALL, new Id("sasl", user)));
922           }
923         }
924         if (!groups.isEmpty()) {
925           LOG.warn("Znode ACL setting for group " + groups
926               + " is skipped, Zookeeper doesn't support this feature presently.");
927         }
928       }
929       
930       
931       if (zkw.isClientReadable(node)) {
932         acls.addAll(Ids.CREATOR_ALL_ACL);
933         acls.addAll(Ids.READ_ACL_UNSAFE);
934       } else {
935         acls.addAll(Ids.CREATOR_ALL_ACL);
936       }
937       return acls;
938     } else {
939       return Ids.OPEN_ACL_UNSAFE;
940     }
941   }
942 
943   
944   
945   
946 
947   
948 
949 
950 
951 
952 
953 
954 
955 
956 
957 
958 
959 
960 
961 
962 
963 
964 
965   public static boolean createEphemeralNodeAndWatch(ZooKeeperWatcher zkw,
966       String znode, byte [] data)
967   throws KeeperException {
968     boolean ret = true;
969     try {
970       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
971           CreateMode.EPHEMERAL);
972     } catch (KeeperException.NodeExistsException nee) {
973       ret = false;
974     } catch (InterruptedException e) {
975       LOG.info("Interrupted", e);
976       Thread.currentThread().interrupt();
977     }
978     if(!watchAndCheckExists(zkw, znode)) {
979       
980       return createEphemeralNodeAndWatch(zkw, znode, data);
981     }
982     return ret;
983   }
984 
985   
986 
987 
988 
989 
990 
991 
992 
993 
994 
995 
996 
997 
998 
999 
1000 
1001 
1002 
1003 
1004 
1005   public static boolean createNodeIfNotExistsAndWatch(
1006       ZooKeeperWatcher zkw, String znode, byte [] data)
1007   throws KeeperException {
1008     boolean ret = true;
1009     try {
1010       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1011           CreateMode.PERSISTENT);
1012     } catch (KeeperException.NodeExistsException nee) {
1013       ret = false;
1014     } catch (InterruptedException e) {
1015       zkw.interruptedException(e);
1016       return false;
1017     }
1018     try {
1019       zkw.getRecoverableZooKeeper().exists(znode, zkw);
1020     } catch (InterruptedException e) {
1021       zkw.interruptedException(e);
1022       return false;
1023     }
1024     return ret;
1025   }
1026 
1027   
1028 
1029 
1030 
1031 
1032 
1033 
1034 
1035 
1036 
1037 
1038 
1039 
1040 
1041   public static String createNodeIfNotExistsNoWatch(ZooKeeperWatcher zkw, String znode,
1042       byte[] data, CreateMode createMode) throws KeeperException {
1043 
1044     String createdZNode = null;
1045     try {
1046       createdZNode = zkw.getRecoverableZooKeeper().create(znode, data,
1047           createACL(zkw, znode), createMode);
1048     } catch (KeeperException.NodeExistsException nee) {
1049       return znode;
1050     } catch (InterruptedException e) {
1051       zkw.interruptedException(e);
1052       return null;
1053     }
1054     return createdZNode;
1055   }
1056 
1057   
1058 
1059 
1060 
1061 
1062 
1063 
1064 
1065 
1066 
1067 
1068 
1069 
1070 
1071 
1072 
1073   public static int createAndWatch(ZooKeeperWatcher zkw,
1074       String znode, byte [] data)
1075   throws KeeperException, KeeperException.NodeExistsException {
1076     try {
1077       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1078           CreateMode.PERSISTENT);
1079       Stat stat = zkw.getRecoverableZooKeeper().exists(znode, zkw);
1080       if (stat == null){
1081         
1082         throw KeeperException.create(KeeperException.Code.SYSTEMERROR,
1083             "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode);
1084       }
1085      return stat.getVersion();
1086     } catch (InterruptedException e) {
1087       zkw.interruptedException(e);
1088       return -1;
1089     }
1090   }
1091 
1092   
1093 
1094 
1095 
1096 
1097 
1098 
1099 
1100 
1101 
1102 
1103 
1104 
1105   public static void asyncCreate(ZooKeeperWatcher zkw,
1106       String znode, byte [] data, final AsyncCallback.StringCallback cb,
1107       final Object ctx) {
1108     zkw.getRecoverableZooKeeper().getZooKeeper().create(znode, data,
1109         createACL(zkw, znode), CreateMode.PERSISTENT, cb, ctx);
1110   }
1111 
1112   
1113 
1114 
1115 
1116 
1117 
1118 
1119 
1120 
1121 
1122   public static void createAndFailSilent(ZooKeeperWatcher zkw,
1123       String znode) throws KeeperException {
1124     createAndFailSilent(zkw, znode, new byte[0]);
1125   }
1126 
1127   
1128 
1129 
1130 
1131 
1132 
1133 
1134 
1135 
1136 
1137 
1138   public static void createAndFailSilent(ZooKeeperWatcher zkw,
1139       String znode, byte[] data)
1140   throws KeeperException {
1141     createAndFailSilent(zkw,
1142         (CreateAndFailSilent)ZKUtilOp.createAndFailSilent(znode, data));
1143   }
1144 
1145   private static void createAndFailSilent(ZooKeeperWatcher zkw, CreateAndFailSilent cafs)
1146   throws KeeperException {
1147     CreateRequest create = (CreateRequest)toZooKeeperOp(zkw, cafs).toRequestRecord();
1148     String znode = create.getPath();
1149     try {
1150       RecoverableZooKeeper zk = zkw.getRecoverableZooKeeper();
1151       if (zk.exists(znode, false) == null) {
1152         zk.create(znode, create.getData(), create.getAcl(), CreateMode.fromFlag(create.getFlags()));
1153       }
1154     } catch(KeeperException.NodeExistsException nee) {
1155     } catch(KeeperException.NoAuthException nee){
1156       try {
1157         if (null == zkw.getRecoverableZooKeeper().exists(znode, false)) {
1158           
1159           throw(nee);
1160         }
1161       } catch (InterruptedException ie) {
1162         zkw.interruptedException(ie);
1163       }
1164     } catch(InterruptedException ie) {
1165       zkw.interruptedException(ie);
1166     }
1167   }
1168 
1169   
1170 
1171 
1172 
1173 
1174 
1175 
1176 
1177 
1178 
1179 
1180   public static void createWithParents(ZooKeeperWatcher zkw, String znode)
1181   throws KeeperException {
1182     createWithParents(zkw, znode, new byte[0]);
1183   }
1184 
1185   
1186 
1187 
1188 
1189 
1190 
1191 
1192 
1193 
1194 
1195 
1196 
1197 
1198   public static void createWithParents(ZooKeeperWatcher zkw, String znode, byte[] data)
1199   throws KeeperException {
1200     try {
1201       if(znode == null) {
1202         return;
1203       }
1204       zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode),
1205           CreateMode.PERSISTENT);
1206     } catch(KeeperException.NodeExistsException nee) {
1207       return;
1208     } catch(KeeperException.NoNodeException nne) {
1209       createWithParents(zkw, getParent(znode));
1210       createWithParents(zkw, znode, data);
1211     } catch(InterruptedException ie) {
1212       zkw.interruptedException(ie);
1213     }
1214   }
1215 
1216   
1217   
1218   
1219 
1220   
1221 
1222 
1223   public static void deleteNode(ZooKeeperWatcher zkw, String node)
1224   throws KeeperException {
1225     deleteNode(zkw, node, -1);
1226   }
1227 
1228   
1229 
1230 
1231 
1232   public static boolean deleteNode(ZooKeeperWatcher zkw, String node,
1233       int version)
1234   throws KeeperException {
1235     try {
1236       zkw.getRecoverableZooKeeper().delete(node, version);
1237       return true;
1238     } catch(KeeperException.BadVersionException bve) {
1239       return false;
1240     } catch(InterruptedException ie) {
1241       zkw.interruptedException(ie);
1242       return false;
1243     }
1244   }
1245 
1246   
1247 
1248 
1249 
1250 
1251 
1252   public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)
1253   throws KeeperException {
1254     deleteNodeFailSilent(zkw,
1255       (DeleteNodeFailSilent)ZKUtilOp.deleteNodeFailSilent(node));
1256   }
1257 
1258   private static void deleteNodeFailSilent(ZooKeeperWatcher zkw,
1259       DeleteNodeFailSilent dnfs) throws KeeperException {
1260     DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord();
1261     try {
1262       zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion());
1263     } catch(KeeperException.NoNodeException nne) {
1264     } catch(InterruptedException ie) {
1265       zkw.interruptedException(ie);
1266     }
1267   }
1268 
1269 
1270   
1271 
1272 
1273 
1274 
1275 
1276 
1277 
1278   public static void deleteNodeRecursively(ZooKeeperWatcher zkw, String node)
1279   throws KeeperException {
1280     deleteNodeRecursivelyMultiOrSequential(zkw, true, node);
1281   }
1282 
1283   
1284 
1285 
1286 
1287 
1288 
1289 
1290 
1291 
1292 
1293 
1294   public static void deleteChildrenRecursively(ZooKeeperWatcher zkw, String node)
1295       throws KeeperException {
1296     deleteChildrenRecursivelyMultiOrSequential(zkw, true, node);
1297   }
1298 
1299   
1300 
1301 
1302 
1303 
1304 
1305 
1306 
1307 
1308 
1309 
1310 
1311 
1312 
1313 
1314 
1315 
1316 
1317 
1318 
1319 
1320 
1321 
1322 
1323 
1324 
1325 
1326 
1327 
1328 
1329 
1330 
1331 
1332 
1333   public static void deleteChildrenRecursivelyMultiOrSequential(
1334       ZooKeeperWatcher zkw, boolean runSequentialOnMultiFailure,
1335       String... pathRoots) throws KeeperException {
1336     if (pathRoots == null || pathRoots.length <= 0) {
1337       LOG.warn("Given path is not valid!");
1338       return;
1339     }
1340     List<ZKUtilOp> ops = new ArrayList<ZKUtil.ZKUtilOp>();
1341     for (String eachRoot : pathRoots) {
1342       List<String> children = listChildrenBFSNoWatch(zkw, eachRoot);
1343       
1344       for (int i = children.size() - 1; i >= 0; --i) {
1345         ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
1346       }
1347     }
1348     
1349     if (ops.size() > 0) {
1350       multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
1351     }
1352   }
1353 
1354   
1355 
1356 
1357 
1358 
1359 
1360 
1361 
1362 
1363 
1364 
1365 
1366 
1367 
1368 
1369 
1370 
1371 
1372 
1373 
1374 
1375 
1376 
1377 
1378 
1379 
1380 
1381 
1382 
1383 
1384 
1385 
1386 
1387 
1388   public static void deleteNodeRecursivelyMultiOrSequential(ZooKeeperWatcher zkw,
1389       boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException {
1390     if (pathRoots == null || pathRoots.length <= 0) {
1391       LOG.warn("Given path is not valid!");
1392       return;
1393     }
1394     List<ZKUtilOp> ops = new ArrayList<ZKUtil.ZKUtilOp>();
1395     for (String eachRoot : pathRoots) {
1396       
1397       
1398       List<String> children = listChildrenBFSAndWatchThem(zkw, eachRoot);
1399       
1400       for (int i = children.size() - 1; i >= 0; --i) {
1401         ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i)));
1402       }
1403       try {
1404         if (zkw.getRecoverableZooKeeper().exists(eachRoot, zkw) != null) {
1405           ops.add(ZKUtilOp.deleteNodeFailSilent(eachRoot));
1406         }
1407       } catch (InterruptedException e) {
1408         zkw.interruptedException(e);
1409       }
1410     }
1411     
1412     if (ops.size() > 0) {
1413       multiOrSequential(zkw, ops, runSequentialOnMultiFailure);
1414     }
1415   }
1416 
1417   
1418 
1419 
1420 
1421 
1422 
1423 
1424 
1425 
1426 
1427 
1428 
1429 
1430   private static List<String> listChildrenBFSNoWatch(ZooKeeperWatcher zkw,
1431       final String znode) throws KeeperException {
1432     Deque<String> queue = new LinkedList<String>();
1433     List<String> tree = new ArrayList<String>();
1434     queue.add(znode);
1435     while (true) {
1436       String node = queue.pollFirst();
1437       if (node == null) {
1438         break;
1439       }
1440       List<String> children = listChildrenNoWatch(zkw, node);
1441       if (children == null) {
1442         continue;
1443       }
1444       for (final String child : children) {
1445         final String childPath = node + "/" + child;
1446         queue.add(childPath);
1447         tree.add(childPath);
1448       }
1449     }
1450     return tree;
1451   }
1452 
1453   
1454 
1455 
1456 
1457 
1458 
1459 
1460 
1461 
1462 
1463 
1464 
1465 
1466   private static List<String> listChildrenBFSAndWatchThem(ZooKeeperWatcher zkw, final String znode)
1467       throws KeeperException {
1468     Deque<String> queue = new LinkedList<String>();
1469     List<String> tree = new ArrayList<String>();
1470     queue.add(znode);
1471     while (true) {
1472       String node = queue.pollFirst();
1473       if (node == null) {
1474         break;
1475       }
1476       List<String> children = listChildrenAndWatchThem(zkw, node);
1477       if (children == null) {
1478         continue;
1479       }
1480       for (final String child : children) {
1481         final String childPath = node + "/" + child;
1482         queue.add(childPath);
1483         tree.add(childPath);
1484       }
1485     }
1486     return tree;
1487   }
1488 
1489   
1490 
1491 
1492 
1493 
1494   public abstract static class ZKUtilOp {
1495     private String path;
1496 
1497     private ZKUtilOp(String path) {
1498       this.path = path;
1499     }
1500 
1501     
1502 
1503 
1504     public static ZKUtilOp createAndFailSilent(String path, byte[] data) {
1505       return new CreateAndFailSilent(path, data);
1506     }
1507 
1508     
1509 
1510 
1511     public static ZKUtilOp deleteNodeFailSilent(String path) {
1512       return new DeleteNodeFailSilent(path);
1513     }
1514 
1515     
1516 
1517 
1518     public static ZKUtilOp setData(String path, byte [] data) {
1519       return new SetData(path, data);
1520     }
1521 
1522     
1523 
1524 
1525     public String getPath() {
1526       return path;
1527     }
1528 
1529     
1530 
1531 
1532 
1533     public static class CreateAndFailSilent extends ZKUtilOp {
1534       private byte [] data;
1535 
1536       private CreateAndFailSilent(String path, byte [] data) {
1537         super(path);
1538         this.data = data;
1539       }
1540 
1541       public byte[] getData() {
1542         return data;
1543       }
1544 
1545       @Override
1546       public boolean equals(Object o) {
1547         if (this == o) return true;
1548         if (!(o instanceof CreateAndFailSilent)) return false;
1549 
1550         CreateAndFailSilent op = (CreateAndFailSilent) o;
1551         return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
1552       }
1553 
1554       @Override
1555       public int hashCode() {
1556         int ret = 17 + getPath().hashCode() * 31;
1557         return ret * 31 + Bytes.hashCode(data);
1558       }
1559     }
1560 
1561     
1562 
1563 
1564 
1565     public static class DeleteNodeFailSilent extends ZKUtilOp {
1566       private DeleteNodeFailSilent(String path) {
1567         super(path);
1568       }
1569 
1570       @Override
1571       public boolean equals(Object o) {
1572         if (this == o) return true;
1573         if (!(o instanceof DeleteNodeFailSilent)) return false;
1574 
1575         return super.equals(o);
1576       }
1577 
1578       @Override
1579       public int hashCode() {
1580         return getPath().hashCode();
1581       }
1582     }
1583 
1584     
1585 
1586 
1587     public static class SetData extends ZKUtilOp {
1588       private byte [] data;
1589 
1590       private SetData(String path, byte [] data) {
1591         super(path);
1592         this.data = data;
1593       }
1594 
1595       public byte[] getData() {
1596         return data;
1597       }
1598 
1599       @Override
1600       public boolean equals(Object o) {
1601         if (this == o) return true;
1602         if (!(o instanceof SetData)) return false;
1603 
1604         SetData op = (SetData) o;
1605         return getPath().equals(op.getPath()) && Arrays.equals(data, op.data);
1606       }
1607 
1608       @Override
1609       public int hashCode() {
1610         int ret = getPath().hashCode();
1611         return ret * 31 + Bytes.hashCode(data);
1612       }
1613     }
1614   }
1615 
1616   
1617 
1618 
1619   private static Op toZooKeeperOp(ZooKeeperWatcher zkw, ZKUtilOp op)
1620   throws UnsupportedOperationException {
1621     if(op == null) return null;
1622 
1623     if (op instanceof CreateAndFailSilent) {
1624       CreateAndFailSilent cafs = (CreateAndFailSilent)op;
1625       return Op.create(cafs.getPath(), cafs.getData(), createACL(zkw, cafs.getPath()),
1626         CreateMode.PERSISTENT);
1627     } else if (op instanceof DeleteNodeFailSilent) {
1628       DeleteNodeFailSilent dnfs = (DeleteNodeFailSilent)op;
1629       return Op.delete(dnfs.getPath(), -1);
1630     } else if (op instanceof SetData) {
1631       SetData sd = (SetData)op;
1632       return Op.setData(sd.getPath(), sd.getData(), -1);
1633     } else {
1634       throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1635         + op.getClass().getName());
1636     }
1637   }
1638 
1639   
1640 
1641 
1642 
1643 
1644 
1645 
1646 
1647 
1648 
1649 
1650 
1651 
1652 
1653 
1654 
1655 
1656 
1657 
1658 
1659 
1660   public static void multiOrSequential(ZooKeeperWatcher zkw, List<ZKUtilOp> ops,
1661       boolean runSequentialOnMultiFailure) throws KeeperException {
1662     if (ops == null) return;
1663     boolean useMulti = zkw.getConfiguration().getBoolean(HConstants.ZOOKEEPER_USEMULTI, false);
1664 
1665     if (useMulti) {
1666       List<Op> zkOps = new LinkedList<Op>();
1667       for (ZKUtilOp op : ops) {
1668         zkOps.add(toZooKeeperOp(zkw, op));
1669       }
1670       try {
1671         zkw.getRecoverableZooKeeper().multi(zkOps);
1672       } catch (KeeperException ke) {
1673        switch (ke.code()) {
1674          case NODEEXISTS:
1675          case NONODE:
1676          case BADVERSION:
1677          case NOAUTH:
1678            
1679            
1680            if (runSequentialOnMultiFailure) {
1681              LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "."
1682                + "  Attempting to run operations sequentially because"
1683                + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + ".");
1684              processSequentially(zkw, ops);
1685              break;
1686            }
1687           default:
1688             throw ke;
1689         }
1690       } catch (InterruptedException ie) {
1691         zkw.interruptedException(ie);
1692       }
1693     } else {
1694       
1695       processSequentially(zkw, ops);
1696     }
1697 
1698   }
1699 
1700   private static void processSequentially(ZooKeeperWatcher zkw, List<ZKUtilOp> ops)
1701       throws KeeperException, NoNodeException {
1702     for (ZKUtilOp op : ops) {
1703       if (op instanceof CreateAndFailSilent) {
1704         createAndFailSilent(zkw, (CreateAndFailSilent) op);
1705       } else if (op instanceof DeleteNodeFailSilent) {
1706         deleteNodeFailSilent(zkw, (DeleteNodeFailSilent) op);
1707       } else if (op instanceof SetData) {
1708         setData(zkw, (SetData) op);
1709       } else {
1710         throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1711             + op.getClass().getName());
1712       }
1713     }
1714   }
1715 
1716   
1717   
1718   
1719 
1720   
1721   public static String dump(ZooKeeperWatcher zkw) {
1722     StringBuilder sb = new StringBuilder();
1723     try {
1724       sb.append("HBase is rooted at ").append(zkw.baseZNode);
1725       sb.append("\nActive master address: ");
1726       try {
1727         sb.append(MasterAddressTracker.getMasterAddress(zkw));
1728       } catch (IOException e) {
1729         sb.append("<<FAILED LOOKUP: " + e.getMessage() + ">>");
1730       }
1731       sb.append("\nBackup master addresses:");
1732       for (String child : listChildrenNoWatch(zkw,
1733                                               zkw.backupMasterAddressesZNode)) {
1734         sb.append("\n ").append(child);
1735       }
1736       sb.append("\nRegion server holding hbase:meta: "
1737         + new MetaTableLocator().getMetaRegionLocation(zkw));
1738       Configuration conf = HBaseConfiguration.create();
1739       int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
1740                HConstants.DEFAULT_META_REPLICA_NUM);
1741       for (int i = 1; i < numMetaReplicas; i++) {
1742         sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
1743                     + new MetaTableLocator().getMetaRegionLocation(zkw, i));
1744       }
1745       sb.append("\nRegion servers:");
1746       for (String child : listChildrenNoWatch(zkw, zkw.rsZNode)) {
1747         sb.append("\n ").append(child);
1748       }
1749       try {
1750         getReplicationZnodesDump(zkw, sb);
1751       } catch (KeeperException ke) {
1752         LOG.warn("Couldn't get the replication znode dump", ke);
1753       }
1754       sb.append("\nQuorum Server Statistics:");
1755       String[] servers = zkw.getQuorum().split(",");
1756       for (String server : servers) {
1757         sb.append("\n ").append(server);
1758         try {
1759           String[] stat = getServerStats(server, ZKUtil.zkDumpConnectionTimeOut);
1760 
1761           if (stat == null) {
1762             sb.append("[Error] invalid quorum server: " + server);
1763             break;
1764           }
1765 
1766           for (String s : stat) {
1767             sb.append("\n  ").append(s);
1768           }
1769         } catch (Exception e) {
1770           sb.append("\n  ERROR: ").append(e.getMessage());
1771         }
1772       }
1773     } catch (KeeperException ke) {
1774       sb.append("\nFATAL ZooKeeper Exception!\n");
1775       sb.append("\n" + ke.getMessage());
1776     }
1777     return sb.toString();
1778   }
1779 
1780   
1781 
1782 
1783 
1784 
1785 
1786   private static void getReplicationZnodesDump(ZooKeeperWatcher zkw, StringBuilder sb)
1787       throws KeeperException {
1788     String replicationZNodeName = zkw.getConfiguration().get("zookeeper.znode.replication",
1789       "replication");
1790     String replicationZnode = joinZNode(zkw.baseZNode, replicationZNodeName);
1791     if (ZKUtil.checkExists(zkw, replicationZnode) == -1) return;
1792     
1793     sb.append("\n").append(replicationZnode).append(": ");
1794     List<String> children = ZKUtil.listChildrenNoWatch(zkw, replicationZnode);
1795     for (String child : children) {
1796       String znode = joinZNode(replicationZnode, child);
1797       if (child.equals(zkw.getConfiguration().get("zookeeper.znode.replication.peers", "peers"))) {
1798         appendPeersZnodes(zkw, znode, sb);
1799       } else if (child.equals(zkw.getConfiguration().
1800           get("zookeeper.znode.replication.rs", "rs"))) {
1801         appendRSZnodes(zkw, znode, sb);
1802       }
1803     }
1804   }
1805 
1806   private static void appendRSZnodes(ZooKeeperWatcher zkw, String znode, StringBuilder sb)
1807       throws KeeperException {
1808     List<String> stack = new LinkedList<String>();
1809     stack.add(znode);
1810     do {
1811       String znodeToProcess = stack.remove(stack.size() - 1);
1812       sb.append("\n").append(znodeToProcess).append(": ");
1813       byte[] data;
1814       try {
1815         data = ZKUtil.getData(zkw, znodeToProcess);
1816       } catch (InterruptedException e) {
1817         zkw.interruptedException(e);
1818         return;
1819       }
1820       if (data != null && data.length > 0) { 
1821         long position = 0;
1822         try {
1823           position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess));
1824           sb.append(position);
1825         } catch (DeserializationException ignored) {
1826         } catch (InterruptedException e) {
1827           zkw.interruptedException(e);
1828           return;
1829         }
1830       }
1831       for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
1832         stack.add(ZKUtil.joinZNode(znodeToProcess, zNodeChild));
1833       }
1834     } while (stack.size() > 0);
1835   }
1836 
1837   private static void appendPeersZnodes(ZooKeeperWatcher zkw, String peersZnode,
1838     StringBuilder sb) throws KeeperException {
1839     int pblen = ProtobufUtil.lengthOfPBMagic();
1840     sb.append("\n").append(peersZnode).append(": ");
1841     for (String peerIdZnode : ZKUtil.listChildrenNoWatch(zkw, peersZnode)) {
1842       String znodeToProcess = ZKUtil.joinZNode(peersZnode, peerIdZnode);
1843       byte[] data;
1844       try {
1845         data = ZKUtil.getData(zkw, znodeToProcess);
1846       } catch (InterruptedException e) {
1847         zkw.interruptedException(e);
1848         return;
1849       }
1850       
1851       try {
1852         ZooKeeperProtos.ReplicationPeer.Builder builder =
1853           ZooKeeperProtos.ReplicationPeer.newBuilder();
1854         ProtobufUtil.mergeFrom(builder, data, pblen, data.length - pblen);
1855         String clusterKey = builder.getClusterkey();
1856         sb.append("\n").append(znodeToProcess).append(": ").append(clusterKey);
1857         
1858         appendPeerState(zkw, znodeToProcess, sb);
1859       } catch (IOException ipbe) {
1860         LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
1861       }
1862     }
1863   }
1864 
1865   private static void appendPeerState(ZooKeeperWatcher zkw, String znodeToProcess,
1866       StringBuilder sb) throws KeeperException, InvalidProtocolBufferException {
1867     String peerState = zkw.getConfiguration().get("zookeeper.znode.replication.peers.state",
1868       "peer-state");
1869     int pblen = ProtobufUtil.lengthOfPBMagic();
1870     for (String child : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) {
1871       if (!child.equals(peerState)) continue;
1872       String peerStateZnode = ZKUtil.joinZNode(znodeToProcess, child);
1873       sb.append("\n").append(peerStateZnode).append(": ");
1874       byte[] peerStateData;
1875       try {
1876         peerStateData = ZKUtil.getData(zkw, peerStateZnode);
1877         ZooKeeperProtos.ReplicationState.Builder builder =
1878             ZooKeeperProtos.ReplicationState.newBuilder();
1879         ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen);
1880         sb.append(builder.getState().name());
1881       } catch (IOException ipbe) {
1882         LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe);
1883       } catch (InterruptedException e) {
1884         zkw.interruptedException(e);
1885         return;
1886       }
1887     }
1888   }
1889 
1890   
1891 
1892 
1893 
1894 
1895 
1896 
1897 
1898   public static String[] getServerStats(String server, int timeout)
1899   throws IOException {
1900     String[] sp = server.split(":");
1901     if (sp == null || sp.length == 0) {
1902       return null;
1903     }
1904 
1905     String host = sp[0];
1906     int port = sp.length > 1 ? Integer.parseInt(sp[1])
1907         : HConstants.DEFAULT_ZOOKEPER_CLIENT_PORT;
1908 
1909     Socket socket = new Socket();
1910     InetSocketAddress sockAddr = new InetSocketAddress(host, port);
1911     socket.connect(sockAddr, timeout);
1912 
1913     socket.setSoTimeout(timeout);
1914     PrintWriter out = new PrintWriter(socket.getOutputStream(), true);
1915     BufferedReader in = new BufferedReader(new InputStreamReader(
1916       socket.getInputStream()));
1917     out.println("stat");
1918     out.flush();
1919     ArrayList<String> res = new ArrayList<String>();
1920     while (true) {
1921       String line = in.readLine();
1922       if (line != null) {
1923         res.add(line);
1924       } else {
1925         break;
1926       }
1927     }
1928     socket.close();
1929     return res.toArray(new String[res.size()]);
1930   }
1931 
1932   private static void logRetrievedMsg(final ZooKeeperWatcher zkw,
1933       final String znode, final byte [] data, final boolean watcherSet) {
1934     if (!LOG.isTraceEnabled()) return;
1935     LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) +
1936       " byte(s) of data from znode " + znode +
1937       (watcherSet? " and set watcher; ": "; data=") +
1938       (data == null? "null": data.length == 0? "empty": (
1939           znode.startsWith(zkw.assignmentZNode)?
1940             ZKAssign.toString(data): 
1941           znode.startsWith(ZooKeeperWatcher.META_ZNODE_PREFIX)?
1942             getServerNameOrEmptyString(data):
1943           znode.startsWith(zkw.backupMasterAddressesZNode)?
1944             getServerNameOrEmptyString(data):
1945           StringUtils.abbreviate(Bytes.toStringBinary(data), 32)))));
1946   }
1947 
1948   private static String getServerNameOrEmptyString(final byte [] data) {
1949     try {
1950       return ServerName.parseFrom(data).toString();
1951     } catch (DeserializationException e) {
1952       return "";
1953     }
1954   }
1955 
1956   
1957 
1958 
1959 
1960   public static void waitForBaseZNode(Configuration conf) throws IOException {
1961     LOG.info("Waiting until the base znode is available");
1962     String parentZNode = conf.get(HConstants.ZOOKEEPER_ZNODE_PARENT,
1963         HConstants.DEFAULT_ZOOKEEPER_ZNODE_PARENT);
1964     ZooKeeper zk = new ZooKeeper(ZKConfig.getZKQuorumServersString(conf),
1965         conf.getInt(HConstants.ZK_SESSION_TIMEOUT,
1966         HConstants.DEFAULT_ZK_SESSION_TIMEOUT), EmptyWatcher.instance);
1967 
1968     final int maxTimeMs = 10000;
1969     final int maxNumAttempts = maxTimeMs / HConstants.SOCKET_RETRY_WAIT_MS;
1970 
1971     KeeperException keeperEx = null;
1972     try {
1973       try {
1974         for (int attempt = 0; attempt < maxNumAttempts; ++attempt) {
1975           try {
1976             if (zk.exists(parentZNode, false) != null) {
1977               LOG.info("Parent znode exists: " + parentZNode);
1978               keeperEx = null;
1979               break;
1980             }
1981           } catch (KeeperException e) {
1982             keeperEx = e;
1983           }
1984           Threads.sleepWithoutInterrupt(HConstants.SOCKET_RETRY_WAIT_MS);
1985         }
1986       } finally {
1987         zk.close();
1988       }
1989     } catch (InterruptedException ex) {
1990       Thread.currentThread().interrupt();
1991     }
1992 
1993     if (keeperEx != null) {
1994       throw new IOException(keeperEx);
1995     }
1996   }
1997 
1998 
1999   public static byte[] blockUntilAvailable(
2000     final ZooKeeperWatcher zkw, final String znode, final long timeout)
2001     throws InterruptedException {
2002     if (timeout < 0) throw new IllegalArgumentException();
2003     if (zkw == null) throw new IllegalArgumentException();
2004     if (znode == null) throw new IllegalArgumentException();
2005 
2006     byte[] data = null;
2007     boolean finished = false;
2008     final long endTime = System.currentTimeMillis() + timeout;
2009     while (!finished) {
2010       try {
2011         data = ZKUtil.getData(zkw, znode);
2012       } catch(KeeperException e) {
2013         if (e instanceof KeeperException.SessionExpiredException
2014             || e instanceof KeeperException.AuthFailedException) {
2015           
2016           throw new InterruptedException("interrupted due to " + e);
2017         }
2018         LOG.warn("Unexpected exception handling blockUntilAvailable", e);
2019       }
2020 
2021       if (data == null && (System.currentTimeMillis() +
2022         HConstants.SOCKET_RETRY_WAIT_MS < endTime)) {
2023         Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
2024       } else {
2025         finished = true;
2026       }
2027     }
2028 
2029     return data;
2030   }
2031 
2032 
2033   
2034 
2035 
2036 
2037 
2038 
2039   public static KeeperException convert(final DeserializationException e) {
2040     KeeperException ke = new KeeperException.DataInconsistencyException();
2041     ke.initCause(e);
2042     return ke;
2043   }
2044 
2045   
2046 
2047 
2048 
2049   public static void logZKTree(ZooKeeperWatcher zkw, String root) {
2050     if (!LOG.isDebugEnabled()) return;
2051     LOG.debug("Current zk system:");
2052     String prefix = "|-";
2053     LOG.debug(prefix + root);
2054     try {
2055       logZKTree(zkw, root, prefix);
2056     } catch (KeeperException e) {
2057       throw new RuntimeException(e);
2058     }
2059   }
2060 
2061   
2062 
2063 
2064 
2065 
2066   protected static void logZKTree(ZooKeeperWatcher zkw, String root, String prefix)
2067       throws KeeperException {
2068     List<String> children = ZKUtil.listChildrenNoWatch(zkw, root);
2069     if (children == null) return;
2070     for (String child : children) {
2071       LOG.debug(prefix + child);
2072       String node = ZKUtil.joinZNode(root.equals("/") ? "" : root, child);
2073       logZKTree(zkw, node, prefix + "---");
2074     }
2075   }
2076 
2077   
2078 
2079 
2080 
2081 
2082   public static byte[] positionToByteArray(final long position) {
2083     byte[] bytes = ZooKeeperProtos.ReplicationHLogPosition.newBuilder().setPosition(position)
2084         .build().toByteArray();
2085     return ProtobufUtil.prependPBMagic(bytes);
2086   }
2087 
2088   
2089 
2090 
2091 
2092 
2093   public static long parseWALPositionFrom(final byte[] bytes) throws DeserializationException {
2094     if (bytes == null) {
2095       throw new DeserializationException("Unable to parse null WAL position.");
2096     }
2097     if (ProtobufUtil.isPBMagicPrefix(bytes)) {
2098       int pblen = ProtobufUtil.lengthOfPBMagic();
2099       ZooKeeperProtos.ReplicationHLogPosition.Builder builder =
2100           ZooKeeperProtos.ReplicationHLogPosition.newBuilder();
2101       ZooKeeperProtos.ReplicationHLogPosition position;
2102       try {
2103         ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
2104         position = builder.build();
2105       } catch (IOException e) {
2106         throw new DeserializationException(e);
2107       }
2108       return position.getPosition();
2109     } else {
2110       if (bytes.length > 0) {
2111         return Bytes.toLong(bytes);
2112       }
2113       return 0;
2114     }
2115   }
2116 
2117   
2118 
2119 
2120 
2121 
2122 
2123 
2124   public static byte[] regionSequenceIdsToByteArray(final Long regionLastFlushedSequenceId,
2125       final Map<byte[], Long> storeSequenceIds) {
2126     ClusterStatusProtos.RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
2127         ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
2128     ClusterStatusProtos.StoreSequenceId.Builder storeSequenceIdBuilder =
2129         ClusterStatusProtos.StoreSequenceId.newBuilder();
2130     if (storeSequenceIds != null) {
2131       for (Map.Entry<byte[], Long> e : storeSequenceIds.entrySet()){
2132         byte[] columnFamilyName = e.getKey();
2133         Long curSeqId = e.getValue();
2134         storeSequenceIdBuilder.setFamilyName(ByteStringer.wrap(columnFamilyName));
2135         storeSequenceIdBuilder.setSequenceId(curSeqId);
2136         regionSequenceIdsBuilder.addStoreSequenceId(storeSequenceIdBuilder.build());
2137         storeSequenceIdBuilder.clear();
2138       }
2139     }
2140     regionSequenceIdsBuilder.setLastFlushedSequenceId(regionLastFlushedSequenceId);
2141     byte[] result = regionSequenceIdsBuilder.build().toByteArray();
2142     return ProtobufUtil.prependPBMagic(result);
2143   }
2144 
2145   
2146 
2147 
2148 
2149 
2150   public static RegionStoreSequenceIds parseRegionStoreSequenceIds(final byte[] bytes)
2151       throws DeserializationException {
2152     if (bytes == null || !ProtobufUtil.isPBMagicPrefix(bytes)) {
2153       throw new DeserializationException("Unable to parse RegionStoreSequenceIds.");
2154     }
2155     RegionStoreSequenceIds.Builder regionSequenceIdsBuilder =
2156         ClusterStatusProtos.RegionStoreSequenceIds.newBuilder();
2157     int pblen = ProtobufUtil.lengthOfPBMagic();
2158     RegionStoreSequenceIds storeIds = null;
2159     try {
2160       ProtobufUtil.mergeFrom(regionSequenceIdsBuilder, bytes, pblen, bytes.length - pblen);
2161       storeIds = regionSequenceIdsBuilder.build();
2162     } catch (IOException e) {
2163       throw new DeserializationException(e);
2164     }
2165     return storeIds;
2166   }
2167 }