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