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