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