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