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