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