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