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