2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.zookeeper
;
20 import java
.io
.IOException
;
21 import java
.util
.ArrayList
;
22 import java
.util
.Arrays
;
23 import java
.util
.Collections
;
24 import java
.util
.Deque
;
25 import java
.util
.Iterator
;
26 import java
.util
.LinkedList
;
27 import java
.util
.List
;
28 import java
.util
.stream
.Collectors
;
29 import org
.apache
.commons
.lang3
.StringUtils
;
30 import org
.apache
.hadoop
.conf
.Configuration
;
31 import org
.apache
.hadoop
.hbase
.HConstants
;
32 import org
.apache
.hadoop
.hbase
.exceptions
.DeserializationException
;
33 import org
.apache
.hadoop
.hbase
.util
.Bytes
;
34 import org
.apache
.hadoop
.hbase
.util
.Threads
;
35 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
.ZKUtilOp
.CreateAndFailSilent
;
36 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
.ZKUtilOp
.DeleteNodeFailSilent
;
37 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
.ZKUtilOp
.SetData
;
38 import org
.apache
.yetus
.audience
.InterfaceAudience
;
39 import org
.apache
.zookeeper
.AsyncCallback
;
40 import org
.apache
.zookeeper
.CreateMode
;
41 import org
.apache
.zookeeper
.KeeperException
;
42 import org
.apache
.zookeeper
.KeeperException
.NoNodeException
;
43 import org
.apache
.zookeeper
.Op
;
44 import org
.apache
.zookeeper
.ZooKeeper
;
45 import org
.apache
.zookeeper
.data
.Stat
;
46 import org
.apache
.zookeeper
.proto
.CreateRequest
;
47 import org
.apache
.zookeeper
.proto
.DeleteRequest
;
48 import org
.apache
.zookeeper
.proto
.SetDataRequest
;
49 import org
.slf4j
.Logger
;
50 import org
.slf4j
.LoggerFactory
;
51 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.ProtobufUtil
;
52 import org
.apache
.hadoop
.hbase
.shaded
.protobuf
.generated
.ReplicationProtos
;
55 * Internal HBase utility class for ZooKeeper.
57 * <p>Contains only static methods and constants.
59 * <p>Methods all throw {@link KeeperException} if there is an unexpected
60 * zookeeper exception, so callers of these methods must handle appropriately.
61 * If ZK is required for the operation, the server will need to be aborted.
63 @InterfaceAudience.Private
64 public final class ZKUtil
{
65 private static final Logger LOG
= LoggerFactory
.getLogger(ZKUtil
.class);
74 * Returns the full path of the immediate parent of the specified node.
75 * @param node path to get parent of
76 * @return parent of path, null if passed the root node or an invalid node
78 public static String
getParent(String node
) {
79 int idx
= node
.lastIndexOf(ZNodePaths
.ZNODE_PATH_SEPARATOR
);
80 return idx
<= 0 ?
null : node
.substring(0, idx
);
84 * Get the name of the current node from the specified fully-qualified path.
85 * @param path fully-qualified path
86 * @return name of the current node
88 public static String
getNodeName(String path
) {
89 return path
.substring(path
.lastIndexOf("/")+1);
93 // Existence checks and watches
97 * Watch the specified znode for delete/create/change events. The watcher is
98 * set whether or not the node exists. If the node already exists, the method
99 * returns true. If the node does not exist, the method returns false.
101 * @param zkw zk reference
102 * @param znode path of node to watch
103 * @return true if znode exists, false if does not exist or error
104 * @throws KeeperException if unexpected zookeeper exception
106 public static boolean watchAndCheckExists(ZKWatcher zkw
, String znode
)
107 throws KeeperException
{
109 Stat s
= zkw
.getRecoverableZooKeeper().exists(znode
, zkw
);
110 boolean exists
= s
!= null;
112 LOG
.debug(zkw
.prefix("Set watcher on existing znode=" + znode
));
114 LOG
.debug(zkw
.prefix("Set watcher on znode that does not yet exist, " + znode
));
117 } catch (KeeperException e
) {
118 LOG
.warn(zkw
.prefix("Unable to set watcher on znode " + znode
), e
);
119 zkw
.keeperException(e
);
121 } catch (InterruptedException e
) {
122 LOG
.warn(zkw
.prefix("Unable to set watcher on znode " + znode
), e
);
123 zkw
.interruptedException(e
);
129 * Watch the specified znode, but only if exists. Useful when watching
130 * for deletions. Uses .getData() (and handles NoNodeException) instead
131 * of .exists() to accomplish this, as .getData() will only set a watch if
133 * @param zkw zk reference
134 * @param znode path of node to watch
135 * @return true if the watch is set, false if node does not exists
136 * @throws KeeperException if unexpected zookeeper exception
138 public static boolean setWatchIfNodeExists(ZKWatcher zkw
, String znode
)
139 throws KeeperException
{
141 zkw
.getRecoverableZooKeeper().getData(znode
, true, null);
143 } catch (NoNodeException e
) {
145 } catch (InterruptedException e
) {
146 LOG
.warn(zkw
.prefix("Unable to set watcher on znode " + znode
), e
);
147 zkw
.interruptedException(e
);
153 * Check if the specified node exists. Sets no watches.
155 * @param zkw zk reference
156 * @param znode path of node to watch
157 * @return version of the node if it exists, -1 if does not exist
158 * @throws KeeperException if unexpected zookeeper exception
160 public static int checkExists(ZKWatcher zkw
, String znode
)
161 throws KeeperException
{
163 Stat s
= zkw
.getRecoverableZooKeeper().exists(znode
, null);
164 return s
!= null ? s
.getVersion() : -1;
165 } catch (KeeperException e
) {
166 LOG
.warn(zkw
.prefix("Unable to set watcher on znode (" + znode
+ ")"), e
);
167 zkw
.keeperException(e
);
169 } catch (InterruptedException e
) {
170 LOG
.warn(zkw
.prefix("Unable to set watcher on znode (" + znode
+ ")"), e
);
171 zkw
.interruptedException(e
);
181 * Lists the children znodes of the specified znode. Also sets a watch on
182 * the specified znode which will capture a NodeDeleted event on the specified
183 * znode as well as NodeChildrenChanged if any children of the specified znode
184 * are created or deleted.
186 * Returns null if the specified node does not exist. Otherwise returns a
187 * list of children of the specified node. If the node exists but it has no
188 * children, an empty list will be returned.
190 * @param zkw zk reference
191 * @param znode path of node to list and watch children of
192 * @return list of children of the specified node, an empty list if the node
193 * exists but has no children, and null if the node does not exist
194 * @throws KeeperException if unexpected zookeeper exception
196 public static List
<String
> listChildrenAndWatchForNewChildren(
197 ZKWatcher zkw
, String znode
)
198 throws KeeperException
{
200 return zkw
.getRecoverableZooKeeper().getChildren(znode
, zkw
);
201 } catch(KeeperException
.NoNodeException ke
) {
202 LOG
.debug(zkw
.prefix("Unable to list children of znode " + znode
+ " " +
203 "because node does not exist (not an error)"));
204 } catch (KeeperException e
) {
205 LOG
.warn(zkw
.prefix("Unable to list children of znode " + znode
+ " "), e
);
206 zkw
.keeperException(e
);
207 } catch (InterruptedException e
) {
208 LOG
.warn(zkw
.prefix("Unable to list children of znode " + znode
+ " "), e
);
209 zkw
.interruptedException(e
);
216 * List all the children of the specified znode, setting a watch for children
217 * changes and also setting a watch on every individual child in order to get
218 * the NodeCreated and NodeDeleted events.
219 * @param zkw zookeeper reference
220 * @param znode node to get children of and watch
221 * @return list of znode names, null if the node doesn't exist
222 * @throws KeeperException if a ZooKeeper operation fails
224 public static List
<String
> listChildrenAndWatchThem(ZKWatcher zkw
,
225 String znode
) throws KeeperException
{
226 List
<String
> children
= listChildrenAndWatchForNewChildren(zkw
, znode
);
227 if (children
== null) {
230 for (String child
: children
) {
231 watchAndCheckExists(zkw
, ZNodePaths
.joinZNode(znode
, child
));
237 * Lists the children of the specified znode without setting any watches.
239 * Sets no watches at all, this method is best effort.
241 * Returns an empty list if the node has no children. Returns null if the
242 * parent node itself does not exist.
244 * @param zkw zookeeper reference
245 * @param znode node to get children
246 * @return list of data of children of specified znode, empty if no children,
247 * null if parent does not exist
248 * @throws KeeperException if unexpected zookeeper exception
250 public static List
<String
> listChildrenNoWatch(ZKWatcher zkw
, String znode
)
251 throws KeeperException
{
252 List
<String
> children
= null;
254 // List the children without watching
255 children
= zkw
.getRecoverableZooKeeper().getChildren(znode
, null);
256 } catch(KeeperException
.NoNodeException nne
) {
258 } catch(InterruptedException ie
) {
259 zkw
.interruptedException(ie
);
265 * Simple class to hold a node path and node data.
269 public static class NodeAndData
{
271 private byte [] data
;
272 public NodeAndData(String node
, byte [] data
) {
276 public String
getNode() {
279 public byte [] getData() {
283 public String
toString() {
286 public boolean isEmpty() {
287 return (data
== null || data
.length
== 0);
292 * Checks if the specified znode has any children. Sets no watches.
294 * Returns true if the node exists and has children. Returns false if the
295 * node does not exist or if the node does not have any children.
297 * Used during master initialization to determine if the master is a
298 * failed-over-to master or the first master during initial cluster startup.
299 * If the directory for regionserver ephemeral nodes is empty then this is
300 * a cluster startup, if not then it is not cluster startup.
302 * @param zkw zk reference
303 * @param znode path of node to check for children of
304 * @return true if node has children, false if not or node does not exist
305 * @throws KeeperException if unexpected zookeeper exception
307 public static boolean nodeHasChildren(ZKWatcher zkw
, String znode
)
308 throws KeeperException
{
310 return !zkw
.getRecoverableZooKeeper().getChildren(znode
, null).isEmpty();
311 } catch(KeeperException
.NoNodeException ke
) {
312 LOG
.debug(zkw
.prefix("Unable to list children of znode " + znode
+
313 " because node does not exist (not an error)"));
315 } catch (KeeperException e
) {
316 LOG
.warn(zkw
.prefix("Unable to list children of znode " + znode
), e
);
317 zkw
.keeperException(e
);
319 } catch (InterruptedException e
) {
320 LOG
.warn(zkw
.prefix("Unable to list children of znode " + znode
), e
);
321 zkw
.interruptedException(e
);
327 * Get the number of children of the specified node.
329 * If the node does not exist or has no children, returns 0.
331 * Sets no watches at all.
333 * @param zkw zk reference
334 * @param znode path of node to count children of
335 * @return number of children of specified node, 0 if none or parent does not
337 * @throws KeeperException if unexpected zookeeper exception
339 public static int getNumberOfChildren(ZKWatcher zkw
, String znode
)
340 throws KeeperException
{
342 Stat stat
= zkw
.getRecoverableZooKeeper().exists(znode
, null);
343 return stat
== null ?
0 : stat
.getNumChildren();
344 } catch(KeeperException e
) {
345 LOG
.warn(zkw
.prefix("Unable to get children of node " + znode
));
346 zkw
.keeperException(e
);
347 } catch(InterruptedException e
) {
348 zkw
.interruptedException(e
);
358 * Get znode data. Does not set a watcher.
360 * @return ZNode data, null if the node does not exist or if there is an error.
362 public static byte [] getData(ZKWatcher zkw
, String znode
)
363 throws KeeperException
, InterruptedException
{
365 byte [] data
= zkw
.getRecoverableZooKeeper().getData(znode
, null, null);
366 logRetrievedMsg(zkw
, znode
, data
, false);
368 } catch (KeeperException
.NoNodeException e
) {
369 LOG
.debug(zkw
.prefix("Unable to get data of znode " + znode
+ " " +
370 "because node does not exist (not an error)"));
372 } catch (KeeperException e
) {
373 LOG
.warn(zkw
.prefix("Unable to get data of znode " + znode
), e
);
374 zkw
.keeperException(e
);
380 * Get the data at the specified znode and set a watch.
382 * Returns the data and sets a watch if the node exists. Returns null and no
383 * watch is set if the node does not exist or there is an exception.
385 * @param zkw zk reference
386 * @param znode path of node
387 * @return data of the specified znode, or null
388 * @throws KeeperException if unexpected zookeeper exception
390 public static byte[] getDataAndWatch(ZKWatcher zkw
, String znode
) throws KeeperException
{
391 return getDataInternal(zkw
, znode
, null, true, true);
395 * Get the data at the specified znode and set a watch.
396 * Returns the data and sets a watch if the node exists. Returns null and no
397 * watch is set if the node does not exist or there is an exception.
399 * @param zkw zk reference
400 * @param znode path of node
401 * @param throwOnInterrupt if false then just interrupt the thread, do not throw exception
402 * @return data of the specified znode, or null
403 * @throws KeeperException if unexpected zookeeper exception
405 public static byte[] getDataAndWatch(ZKWatcher zkw
, String znode
, boolean throwOnInterrupt
)
406 throws KeeperException
{
407 return getDataInternal(zkw
, znode
, null, true, throwOnInterrupt
);
411 * Get the data at the specified znode and set a watch.
413 * Returns the data and sets a watch if the node exists. Returns null and no
414 * watch is set if the node does not exist or there is an exception.
416 * @param zkw zk reference
417 * @param znode path of node
418 * @param stat object to populate the version of the znode
419 * @return data of the specified znode, or null
420 * @throws KeeperException if unexpected zookeeper exception
422 public static byte[] getDataAndWatch(ZKWatcher zkw
, String znode
,
423 Stat stat
) throws KeeperException
{
424 return getDataInternal(zkw
, znode
, stat
, true, true);
427 private static byte[] getDataInternal(ZKWatcher zkw
, String znode
, Stat stat
, boolean watcherSet
,
428 boolean throwOnInterrupt
)
429 throws KeeperException
{
431 byte [] data
= zkw
.getRecoverableZooKeeper().getData(znode
, zkw
, stat
);
432 logRetrievedMsg(zkw
, znode
, data
, watcherSet
);
434 } catch (KeeperException
.NoNodeException e
) {
435 // This log can get pretty annoying when we cycle on 100ms waits.
436 // Enable trace if you really want to see it.
437 LOG
.trace(zkw
.prefix("Unable to get data of znode " + znode
+ " " +
438 "because node does not exist (not an error)"));
440 } catch (KeeperException e
) {
441 LOG
.warn(zkw
.prefix("Unable to get data of znode " + znode
), e
);
442 zkw
.keeperException(e
);
444 } catch (InterruptedException e
) {
445 LOG
.warn(zkw
.prefix("Unable to get data of znode " + znode
), e
);
446 if (throwOnInterrupt
) {
447 zkw
.interruptedException(e
);
449 zkw
.interruptedExceptionNoThrow(e
, true);
456 * Get the data at the specified znode without setting a watch.
458 * Returns the data if the node exists. Returns null if the node does not
461 * Sets the stats of the node in the passed Stat object. Pass a null stat if
464 * @param zkw zk reference
465 * @param znode path of node
466 * @param stat node status to get if node exists
467 * @return data of the specified znode, or null if node does not exist
468 * @throws KeeperException if unexpected zookeeper exception
470 public static byte [] getDataNoWatch(ZKWatcher zkw
, String znode
,
472 throws KeeperException
{
474 byte [] data
= zkw
.getRecoverableZooKeeper().getData(znode
, null, stat
);
475 logRetrievedMsg(zkw
, znode
, data
, false);
477 } catch (KeeperException
.NoNodeException e
) {
478 LOG
.debug(zkw
.prefix("Unable to get data of znode " + znode
+ " " +
479 "because node does not exist (not necessarily an error)"));
481 } catch (KeeperException e
) {
482 LOG
.warn(zkw
.prefix("Unable to get data of znode " + znode
), e
);
483 zkw
.keeperException(e
);
485 } catch (InterruptedException e
) {
486 LOG
.warn(zkw
.prefix("Unable to get data of znode " + znode
), e
);
487 zkw
.interruptedException(e
);
493 * Returns the date of child znodes of the specified znode. Also sets a watch on
494 * the specified znode which will capture a NodeDeleted event on the specified
495 * znode as well as NodeChildrenChanged if any children of the specified znode
496 * are created or deleted.
498 * Returns null if the specified node does not exist. Otherwise returns a
499 * list of children of the specified node. If the node exists but it has no
500 * children, an empty list will be returned.
502 * @param zkw zk reference
503 * @param baseNode path of node to list and watch children of
504 * @return list of data of children of the specified node, an empty list if the node
505 * exists but has no children, and null if the node does not exist
506 * @throws KeeperException if unexpected zookeeper exception
510 public static List
<NodeAndData
> getChildDataAndWatchForNewChildren(ZKWatcher zkw
, String baseNode
)
511 throws KeeperException
{
512 return getChildDataAndWatchForNewChildren(zkw
, baseNode
, true);
516 * Returns the date of child znodes of the specified znode. Also sets a watch on
517 * the specified znode which will capture a NodeDeleted event on the specified
518 * znode as well as NodeChildrenChanged if any children of the specified znode
519 * are created or deleted.
521 * Returns null if the specified node does not exist. Otherwise returns a
522 * list of children of the specified node. If the node exists but it has no
523 * children, an empty list will be returned.
525 * @param zkw zk reference
526 * @param baseNode path of node to list and watch children of
527 * @param throwOnInterrupt if true then just interrupt the thread, do not throw exception
528 * @return list of data of children of the specified node, an empty list if the node
529 * exists but has no children, and null if the node does not exist
530 * @throws KeeperException if unexpected zookeeper exception
534 public static List
<NodeAndData
> getChildDataAndWatchForNewChildren(
535 ZKWatcher zkw
, String baseNode
, boolean throwOnInterrupt
) throws KeeperException
{
537 ZKUtil
.listChildrenAndWatchForNewChildren(zkw
, baseNode
);
539 List
<NodeAndData
> newNodes
= new ArrayList
<>();
540 for (String node
: nodes
) {
541 if (Thread
.interrupted()) {
542 // Partial data should not be processed. Cancel processing by sending empty list.
543 return Collections
.emptyList();
545 String nodePath
= ZNodePaths
.joinZNode(baseNode
, node
);
546 byte[] data
= ZKUtil
.getDataAndWatch(zkw
, nodePath
, throwOnInterrupt
);
547 newNodes
.add(new NodeAndData(nodePath
, data
));
555 * Update the data of an existing node with the expected version to have the
558 * Throws an exception if there is a version mismatch or some other problem.
560 * Sets no watches under any conditions.
562 * @param zkw zk reference
563 * @param znode the path to the ZNode
564 * @param data the data to store in ZooKeeper
565 * @param expectedVersion the expected version
566 * @throws KeeperException if unexpected zookeeper exception
567 * @throws KeeperException.BadVersionException if version mismatch
571 public static void updateExistingNodeData(ZKWatcher zkw
, String znode
, byte[] data
,
572 int expectedVersion
) throws KeeperException
{
574 zkw
.getRecoverableZooKeeper().setData(znode
, data
, expectedVersion
);
575 } catch(InterruptedException ie
) {
576 zkw
.interruptedException(ie
);
585 * Sets the data of the existing znode to be the specified data. Ensures that
586 * the current data has the specified expected version.
588 * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
590 * <p>If their is a version mismatch, method returns null.
592 * <p>No watches are set but setting data will trigger other watchers of this
595 * <p>If there is another problem, a KeeperException will be thrown.
597 * @param zkw zk reference
598 * @param znode path of node
599 * @param data data to set for node
600 * @param expectedVersion version expected when setting data
601 * @return true if data set, false if version mismatch
602 * @throws KeeperException if unexpected zookeeper exception
604 public static boolean setData(ZKWatcher zkw
, String znode
,
605 byte [] data
, int expectedVersion
)
606 throws KeeperException
, KeeperException
.NoNodeException
{
608 return zkw
.getRecoverableZooKeeper().setData(znode
, data
, expectedVersion
) != null;
609 } catch (InterruptedException e
) {
610 zkw
.interruptedException(e
);
616 * Set data into node creating node if it doesn't yet exist.
617 * Does not set watch.
619 * @param zkw zk reference
620 * @param znode path of node
621 * @param data data to set for node
622 * @throws KeeperException if a ZooKeeper operation fails
624 public static void createSetData(final ZKWatcher zkw
, final String znode
, final byte [] data
)
625 throws KeeperException
{
626 if (checkExists(zkw
, znode
) == -1) {
627 ZKUtil
.createWithParents(zkw
, znode
, data
);
629 ZKUtil
.setData(zkw
, znode
, data
);
634 * Sets the data of the existing znode to be the specified data. The node
635 * must exist but no checks are done on the existing data or version.
637 * <p>If the node does not exist, a {@link NoNodeException} will be thrown.
639 * <p>No watches are set but setting data will trigger other watchers of this
642 * <p>If there is another problem, a KeeperException will be thrown.
644 * @param zkw zk reference
645 * @param znode path of node
646 * @param data data to set for node
647 * @throws KeeperException if unexpected zookeeper exception
649 public static void setData(ZKWatcher zkw
, String znode
, byte [] data
)
650 throws KeeperException
, KeeperException
.NoNodeException
{
651 setData(zkw
, (SetData
)ZKUtilOp
.setData(znode
, data
));
654 private static void setData(ZKWatcher zkw
, SetData setData
)
655 throws KeeperException
, KeeperException
.NoNodeException
{
656 SetDataRequest sd
= (SetDataRequest
)toZooKeeperOp(zkw
, setData
).toRequestRecord();
657 setData(zkw
, sd
.getPath(), sd
.getData(), sd
.getVersion());
666 * Set the specified znode to be an ephemeral node carrying the specified
669 * If the node is created successfully, a watcher is also set on the node.
671 * If the node is not created successfully because it already exists, this
672 * method will also set a watcher on the node.
674 * If there is another problem, a KeeperException will be thrown.
676 * @param zkw zk reference
677 * @param znode path of node
678 * @param data data of node
679 * @return true if node created, false if not, watch set in both cases
680 * @throws KeeperException if unexpected zookeeper exception
682 public static boolean createEphemeralNodeAndWatch(ZKWatcher zkw
, String znode
, byte [] data
)
683 throws KeeperException
{
686 zkw
.getRecoverableZooKeeper().create(znode
, data
, zkw
.createACL(znode
),
687 CreateMode
.EPHEMERAL
);
688 } catch (KeeperException
.NodeExistsException nee
) {
690 } catch (InterruptedException e
) {
691 LOG
.info("Interrupted", e
);
692 Thread
.currentThread().interrupt();
694 if(!watchAndCheckExists(zkw
, znode
)) {
695 // It did exist but now it doesn't, try again
696 return createEphemeralNodeAndWatch(zkw
, znode
, data
);
702 * Creates the specified znode to be a persistent node carrying the specified
705 * Returns true if the node was successfully created, false if the node
708 * If the node is created successfully, a watcher is also set on the node.
710 * If the node is not created successfully because it already exists, this
711 * method will also set a watcher on the node but return false.
713 * If there is another problem, a KeeperException will be thrown.
715 * @param zkw zk reference
716 * @param znode path of node
717 * @param data data of node
718 * @return true if node created, false if not, watch set in both cases
719 * @throws KeeperException if unexpected zookeeper exception
721 public static boolean createNodeIfNotExistsAndWatch(
722 ZKWatcher zkw
, String znode
, byte [] data
)
723 throws KeeperException
{
726 zkw
.getRecoverableZooKeeper().create(znode
, data
, zkw
.createACL(znode
),
727 CreateMode
.PERSISTENT
);
728 } catch (KeeperException
.NodeExistsException nee
) {
730 } catch (InterruptedException e
) {
731 zkw
.interruptedException(e
);
735 zkw
.getRecoverableZooKeeper().exists(znode
, zkw
);
736 } catch (InterruptedException e
) {
737 zkw
.interruptedException(e
);
744 * Creates the specified znode with the specified data but does not watch it.
746 * Returns the znode of the newly created node
748 * If there is another problem, a KeeperException will be thrown.
750 * @param zkw zk reference
751 * @param znode path of node
752 * @param data data of node
753 * @param createMode specifying whether the node to be created is ephemeral and/or sequential
754 * @return true name of the newly created znode or null
755 * @throws KeeperException if unexpected zookeeper exception
757 public static String
createNodeIfNotExistsNoWatch(ZKWatcher zkw
, String znode
, byte[] data
,
758 CreateMode createMode
) throws KeeperException
{
760 return zkw
.getRecoverableZooKeeper().create(znode
, data
, zkw
.createACL(znode
), createMode
);
761 } catch (KeeperException
.NodeExistsException nee
) {
763 } catch (InterruptedException e
) {
764 zkw
.interruptedException(e
);
770 * Creates the specified node with the specified data and watches it.
772 * <p>Throws an exception if the node already exists.
774 * <p>The node created is persistent and open access.
776 * <p>Returns the version number of the created node if successful.
778 * @param zkw zk reference
779 * @param znode path of node to create
780 * @param data data of node to create
781 * @return version of node created
782 * @throws KeeperException if unexpected zookeeper exception
783 * @throws KeeperException.NodeExistsException if node already exists
785 public static int createAndWatch(ZKWatcher zkw
,
786 String znode
, byte [] data
)
787 throws KeeperException
, KeeperException
.NodeExistsException
{
789 zkw
.getRecoverableZooKeeper().create(znode
, data
, zkw
.createACL(znode
),
790 CreateMode
.PERSISTENT
);
791 Stat stat
= zkw
.getRecoverableZooKeeper().exists(znode
, zkw
);
793 // Likely a race condition. Someone deleted the znode.
794 throw KeeperException
.create(KeeperException
.Code
.SYSTEMERROR
,
795 "ZK.exists returned null (i.e.: znode does not exist) for znode=" + znode
);
798 return stat
.getVersion();
799 } catch (InterruptedException e
) {
800 zkw
.interruptedException(e
);
806 * Async creates the specified node with the specified data.
808 * <p>Throws an exception if the node already exists.
810 * <p>The node created is persistent and open access.
812 * @param zkw zk reference
813 * @param znode path of node to create
814 * @param data data of node to create
815 * @param cb the callback to use for the creation
816 * @param ctx the context to use for the creation
818 public static void asyncCreate(ZKWatcher zkw
,
819 String znode
, byte [] data
, final AsyncCallback
.StringCallback cb
,
821 zkw
.getRecoverableZooKeeper().getZooKeeper().create(znode
, data
,
822 zkw
.createACL(znode
), CreateMode
.PERSISTENT
, cb
, ctx
);
826 * Creates the specified node, iff the node does not exist. Does not set a
827 * watch and fails silently if the node already exists.
829 * The node created is persistent and open access.
831 * @param zkw zk reference
832 * @param znode path of node
833 * @throws KeeperException if unexpected zookeeper exception
835 public static void createAndFailSilent(ZKWatcher zkw
,
836 String znode
) throws KeeperException
{
837 createAndFailSilent(zkw
, znode
, new byte[0]);
841 * Creates the specified node containing specified data, iff the node does not exist. Does
842 * not set a watch and fails silently if the node already exists.
844 * The node created is persistent and open access.
846 * @param zkw zk reference
847 * @param znode path of node
848 * @param data a byte array data to store in the znode
849 * @throws KeeperException if unexpected zookeeper exception
851 public static void createAndFailSilent(ZKWatcher zkw
,
852 String znode
, byte[] data
)
853 throws KeeperException
{
854 createAndFailSilent(zkw
,
855 (CreateAndFailSilent
)ZKUtilOp
.createAndFailSilent(znode
, data
));
858 private static void createAndFailSilent(ZKWatcher zkw
, CreateAndFailSilent cafs
)
859 throws KeeperException
{
860 CreateRequest create
= (CreateRequest
)toZooKeeperOp(zkw
, cafs
).toRequestRecord();
861 String znode
= create
.getPath();
863 RecoverableZooKeeper zk
= zkw
.getRecoverableZooKeeper();
864 if (zk
.exists(znode
, false) == null) {
865 zk
.create(znode
, create
.getData(), create
.getAcl(), CreateMode
.fromFlag(create
.getFlags()));
867 } catch (KeeperException
.NodeExistsException nee
) {
869 } catch (KeeperException
.NoAuthException nee
) {
871 if (null == zkw
.getRecoverableZooKeeper().exists(znode
, false)) {
872 // If we failed to create the file and it does not already exist.
875 } catch (InterruptedException ie
) {
876 zkw
.interruptedException(ie
);
878 } catch (InterruptedException ie
) {
879 zkw
.interruptedException(ie
);
884 * Creates the specified node and all parent nodes required for it to exist.
886 * No watches are set and no errors are thrown if the node already exists.
888 * The nodes created are persistent and open access.
890 * @param zkw zk reference
891 * @param znode path of node
892 * @throws KeeperException if unexpected zookeeper exception
894 public static void createWithParents(ZKWatcher zkw
, String znode
)
895 throws KeeperException
{
896 createWithParents(zkw
, znode
, new byte[0]);
900 * Creates the specified node and all parent nodes required for it to exist. The creation of
901 * parent znodes is not atomic with the leafe znode creation but the data is written atomically
902 * when the leaf node is created.
904 * No watches are set and no errors are thrown if the node already exists.
906 * The nodes created are persistent and open access.
908 * @param zkw zk reference
909 * @param znode path of node
910 * @throws KeeperException if unexpected zookeeper exception
912 public static void createWithParents(ZKWatcher zkw
, String znode
, byte[] data
)
913 throws KeeperException
{
918 zkw
.getRecoverableZooKeeper().create(znode
, data
, zkw
.createACL(znode
),
919 CreateMode
.PERSISTENT
);
920 } catch(KeeperException
.NodeExistsException nee
) {
922 } catch(KeeperException
.NoNodeException nne
) {
923 createWithParents(zkw
, getParent(znode
));
924 createWithParents(zkw
, znode
, data
);
925 } catch(InterruptedException ie
) {
926 zkw
.interruptedException(ie
);
935 * Delete the specified node. Sets no watches. Throws all exceptions.
937 public static void deleteNode(ZKWatcher zkw
, String node
)
938 throws KeeperException
{
939 deleteNode(zkw
, node
, -1);
943 * Delete the specified node with the specified version. Sets no watches.
944 * Throws all exceptions.
946 public static boolean deleteNode(ZKWatcher zkw
, String node
,
948 throws KeeperException
{
950 zkw
.getRecoverableZooKeeper().delete(node
, version
);
952 } catch(KeeperException
.BadVersionException bve
) {
954 } catch(InterruptedException ie
) {
955 zkw
.interruptedException(ie
);
961 * Deletes the specified node. Fails silent if the node does not exist.
963 * @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
964 * @param node the node to delete
965 * @throws KeeperException if a ZooKeeper operation fails
967 public static void deleteNodeFailSilent(ZKWatcher zkw
, String node
)
968 throws KeeperException
{
969 deleteNodeFailSilent(zkw
,
970 (DeleteNodeFailSilent
)ZKUtilOp
.deleteNodeFailSilent(node
));
973 private static void deleteNodeFailSilent(ZKWatcher zkw
,
974 DeleteNodeFailSilent dnfs
) throws KeeperException
{
975 DeleteRequest delete
= (DeleteRequest
)toZooKeeperOp(zkw
, dnfs
).toRequestRecord();
977 zkw
.getRecoverableZooKeeper().delete(delete
.getPath(), delete
.getVersion());
978 } catch(KeeperException
.NoNodeException nne
) {
979 } catch(InterruptedException ie
) {
980 zkw
.interruptedException(ie
);
986 * Delete the specified node and all of it's children.
988 * If the node does not exist, just returns.
990 * Sets no watches. Throws all exceptions besides dealing with deletion of
993 public static void deleteNodeRecursively(ZKWatcher zkw
, String node
)
994 throws KeeperException
{
995 deleteNodeRecursivelyMultiOrSequential(zkw
, true, node
);
999 * Delete all the children of the specified node but not the node itself.
1001 * Sets no watches. Throws all exceptions besides dealing with deletion of
1004 * @throws KeeperException if a ZooKeeper operation fails
1006 public static void deleteChildrenRecursively(ZKWatcher zkw
, String node
)
1007 throws KeeperException
{
1008 deleteChildrenRecursivelyMultiOrSequential(zkw
, true, node
);
1012 * Delete all the children of the specified node but not the node itself. This
1013 * will first traverse the znode tree for listing the children and then delete
1014 * these znodes using multi-update api or sequential based on the specified
1017 * Sets no watches. Throws all exceptions besides dealing with deletion of
1020 * If the following is true:
1022 * <li>runSequentialOnMultiFailure is true
1024 * on calling multi, we get a ZooKeeper exception that can be handled by a
1025 * sequential call(*), we retry the operations one-by-one (sequentially).
1029 * @param runSequentialOnMultiFailure
1030 * - if true when we get a ZooKeeper exception that could retry the
1031 * operations one-by-one (sequentially)
1033 * - path of the parent node(s)
1034 * @throws KeeperException.NotEmptyException
1035 * if node has children while deleting
1036 * @throws KeeperException
1037 * if unexpected ZooKeeper exception
1038 * @throws IllegalArgumentException
1039 * if an invalid path is specified
1041 public static void deleteChildrenRecursivelyMultiOrSequential(
1042 ZKWatcher zkw
, boolean runSequentialOnMultiFailure
,
1043 String
... pathRoots
) throws KeeperException
{
1044 if (pathRoots
== null || pathRoots
.length
<= 0) {
1045 LOG
.warn("Given path is not valid!");
1048 List
<ZKUtilOp
> ops
= new ArrayList
<>();
1049 for (String eachRoot
: pathRoots
) {
1050 List
<String
> children
= listChildrenBFSNoWatch(zkw
, eachRoot
);
1051 // Delete the leaves first and eventually get rid of the root
1052 for (int i
= children
.size() - 1; i
>= 0; --i
) {
1053 ops
.add(ZKUtilOp
.deleteNodeFailSilent(children
.get(i
)));
1056 submitBatchedMultiOrSequential(zkw
, runSequentialOnMultiFailure
, ops
);
1060 * Delete the specified node and its children. This traverse the
1061 * znode tree for listing the children and then delete
1062 * these znodes including the parent using multi-update api or
1063 * sequential based on the specified configurations.
1065 * Sets no watches. Throws all exceptions besides dealing with deletion of
1068 * If the following is true:
1070 * <li>runSequentialOnMultiFailure is true
1072 * on calling multi, we get a ZooKeeper exception that can be handled by a
1073 * sequential call(*), we retry the operations one-by-one (sequentially).
1077 * @param runSequentialOnMultiFailure
1078 * - if true when we get a ZooKeeper exception that could retry the
1079 * operations one-by-one (sequentially)
1081 * - path of the parent node(s)
1082 * @throws KeeperException.NotEmptyException
1083 * if node has children while deleting
1084 * @throws KeeperException
1085 * if unexpected ZooKeeper exception
1086 * @throws IllegalArgumentException
1087 * if an invalid path is specified
1089 public static void deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw
,
1090 boolean runSequentialOnMultiFailure
, String
... pathRoots
) throws KeeperException
{
1091 if (pathRoots
== null || pathRoots
.length
<= 0) {
1092 LOG
.warn("Given path is not valid!");
1095 List
<ZKUtilOp
> ops
= new ArrayList
<>();
1096 for (String eachRoot
: pathRoots
) {
1097 // ZooKeeper Watches are one time triggers; When children of parent nodes are deleted
1098 // recursively, must set another watch, get notified of delete node
1099 List
<String
> children
= listChildrenBFSAndWatchThem(zkw
, eachRoot
);
1100 // Delete the leaves first and eventually get rid of the root
1101 for (int i
= children
.size() - 1; i
>= 0; --i
) {
1102 ops
.add(ZKUtilOp
.deleteNodeFailSilent(children
.get(i
)));
1105 if (zkw
.getRecoverableZooKeeper().exists(eachRoot
, zkw
) != null) {
1106 ops
.add(ZKUtilOp
.deleteNodeFailSilent(eachRoot
));
1108 } catch (InterruptedException e
) {
1109 zkw
.interruptedException(e
);
1112 submitBatchedMultiOrSequential(zkw
, runSequentialOnMultiFailure
, ops
);
1116 * Chunks the provided {@code ops} when their approximate size exceeds the the configured limit.
1117 * Take caution that this can ONLY be used for operations where atomicity is not important,
1118 * e.g. deletions. It must not be used when atomicity of the operations is critical.
1120 * @param zkw reference to the {@link ZKWatcher} which contains configuration and constants
1121 * @param runSequentialOnMultiFailure if true when we get a ZooKeeper exception that could
1122 * retry the operations one-by-one (sequentially)
1123 * @param ops list of ZKUtilOp {@link ZKUtilOp} to partition while submitting batched multi
1125 * @throws KeeperException unexpected ZooKeeper Exception / Zookeeper unreachable
1127 private static void submitBatchedMultiOrSequential(ZKWatcher zkw
,
1128 boolean runSequentialOnMultiFailure
, List
<ZKUtilOp
> ops
) throws KeeperException
{
1129 // at least one element should exist
1130 if (ops
.isEmpty()) {
1133 final int maxMultiSize
= zkw
.getRecoverableZooKeeper().getMaxMultiSizeLimit();
1134 // Batch up the items to over smashing through jute.maxbuffer with too many Ops.
1135 final List
<List
<ZKUtilOp
>> batchedOps
= partitionOps(ops
, maxMultiSize
);
1136 // Would use forEach() but have to handle KeeperException
1137 for (List
<ZKUtilOp
> batch
: batchedOps
) {
1138 multiOrSequential(zkw
, batch
, runSequentialOnMultiFailure
);
1143 * Partition the list of {@code ops} by size (using {@link #estimateSize(ZKUtilOp)}).
1145 static List
<List
<ZKUtilOp
>> partitionOps(List
<ZKUtilOp
> ops
, int maxPartitionSize
) {
1146 List
<List
<ZKUtilOp
>> partitionedOps
= new ArrayList
<>();
1147 List
<ZKUtilOp
> currentPartition
= new ArrayList
<>();
1148 int currentPartitionSize
= 0;
1149 partitionedOps
.add(currentPartition
);
1150 Iterator
<ZKUtilOp
> iter
= ops
.iterator();
1151 while (iter
.hasNext()) {
1152 ZKUtilOp currentOp
= iter
.next();
1153 int currentOpSize
= estimateSize(currentOp
);
1155 // Roll a new partition if necessary
1156 // If the current partition is empty, put the element in there anyways.
1157 // We can roll a new partition if we get another element
1158 if (!currentPartition
.isEmpty() && currentOpSize
+ currentPartitionSize
> maxPartitionSize
) {
1159 currentPartition
= new ArrayList
<>();
1160 partitionedOps
.add(currentPartition
);
1161 currentPartitionSize
= 0;
1164 // Add the current op to the partition
1165 currentPartition
.add(currentOp
);
1166 // And record its size
1167 currentPartitionSize
+= currentOpSize
;
1169 return partitionedOps
;
1172 static int estimateSize(ZKUtilOp op
) {
1173 return Bytes
.toBytes(op
.getPath()).length
;
1177 * BFS Traversal of all the children under path, with the entries in the list,
1178 * in the same order as that of the traversal. Lists all the children without
1179 * setting any watches.
1185 * @return list of children znodes under the path
1186 * @throws KeeperException
1187 * if unexpected ZooKeeper exception
1189 private static List
<String
> listChildrenBFSNoWatch(ZKWatcher zkw
,
1190 final String znode
) throws KeeperException
{
1191 Deque
<String
> queue
= new LinkedList
<>();
1192 List
<String
> tree
= new ArrayList
<>();
1195 String node
= queue
.pollFirst();
1199 List
<String
> children
= listChildrenNoWatch(zkw
, node
);
1200 if (children
== null) {
1203 for (final String child
: children
) {
1204 final String childPath
= node
+ "/" + child
;
1205 queue
.add(childPath
);
1206 tree
.add(childPath
);
1213 * BFS Traversal of all the children under path, with the entries in the list,
1214 * in the same order as that of the traversal.
1215 * Lists all the children and set watches on to them.
1221 * @return list of children znodes under the path
1222 * @throws KeeperException
1223 * if unexpected ZooKeeper exception
1225 private static List
<String
> listChildrenBFSAndWatchThem(ZKWatcher zkw
, final String znode
)
1226 throws KeeperException
{
1227 Deque
<String
> queue
= new LinkedList
<>();
1228 List
<String
> tree
= new ArrayList
<>();
1231 String node
= queue
.pollFirst();
1235 List
<String
> children
= listChildrenAndWatchThem(zkw
, node
);
1236 if (children
== null) {
1239 for (final String child
: children
) {
1240 final String childPath
= node
+ "/" + child
;
1241 queue
.add(childPath
);
1242 tree
.add(childPath
);
1249 * Represents an action taken by ZKUtil, e.g. createAndFailSilent.
1250 * These actions are higher-level than ZKOp actions, which represent
1251 * individual actions in the ZooKeeper API, like create.
1253 public abstract static class ZKUtilOp
{
1254 private String path
;
1256 @Override public String
toString() {
1257 return this.getClass().getSimpleName() + ", path=" + this.path
;
1260 private ZKUtilOp(String path
) {
1265 * @return a createAndFailSilent ZKUtilOp
1267 public static ZKUtilOp
createAndFailSilent(String path
, byte[] data
) {
1268 return new CreateAndFailSilent(path
, data
);
1272 * @return a deleteNodeFailSilent ZKUtilOP
1274 public static ZKUtilOp
deleteNodeFailSilent(String path
) {
1275 return new DeleteNodeFailSilent(path
);
1279 * @return a setData ZKUtilOp
1281 public static ZKUtilOp
setData(String path
, byte[] data
) {
1282 return new SetData(path
, data
);
1286 * @return a setData ZKUtilOp
1288 public static ZKUtilOp
setData(String path
, byte[] data
, int version
) {
1289 return new SetData(path
, data
, version
);
1293 * @return path to znode where the ZKOp will occur
1295 public String
getPath() {
1300 * ZKUtilOp representing createAndFailSilent in ZooKeeper
1301 * (attempt to create node, ignore error if already exists)
1303 public static final class CreateAndFailSilent
extends ZKUtilOp
{
1304 private byte [] data
;
1306 private CreateAndFailSilent(String path
, byte [] data
) {
1311 public byte[] getData() {
1316 public boolean equals(Object o
) {
1320 if (!(o
instanceof CreateAndFailSilent
)) {
1324 CreateAndFailSilent op
= (CreateAndFailSilent
) o
;
1325 return getPath().equals(op
.getPath()) && Arrays
.equals(data
, op
.data
);
1329 public int hashCode() {
1330 int ret
= 17 + getPath().hashCode() * 31;
1331 return ret
* 31 + Bytes
.hashCode(data
);
1336 * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper
1337 * (attempt to delete node, ignore error if node doesn't exist)
1339 public static final class DeleteNodeFailSilent
extends ZKUtilOp
{
1340 private DeleteNodeFailSilent(String path
) {
1345 public boolean equals(Object o
) {
1349 if (!(o
instanceof DeleteNodeFailSilent
)) {
1353 return super.equals(o
);
1357 public int hashCode() {
1358 return getPath().hashCode();
1363 * ZKUtilOp representing setData in ZooKeeper
1365 public static final class SetData
extends ZKUtilOp
{
1366 private byte[] data
;
1367 private int version
= -1;
1369 private SetData(String path
, byte[] data
) {
1374 private SetData(String path
, byte[] data
, int version
) {
1377 this.version
= version
;
1380 public byte[] getData() {
1384 public int getVersion() {
1389 public boolean equals(Object o
) {
1393 if (!(o
instanceof SetData
)) {
1397 SetData op
= (SetData
) o
;
1398 return getPath().equals(op
.getPath()) && Arrays
.equals(data
, op
.data
)
1399 && getVersion() == op
.getVersion();
1403 public int hashCode() {
1404 int ret
= getPath().hashCode();
1405 ret
= ret
* 31 + Bytes
.hashCode(data
);
1406 return ret
* 31 + Integer
.hashCode(version
);
1412 * Convert from ZKUtilOp to ZKOp
1414 private static Op
toZooKeeperOp(ZKWatcher zkw
, ZKUtilOp op
) throws UnsupportedOperationException
{
1419 if (op
instanceof CreateAndFailSilent
) {
1420 CreateAndFailSilent cafs
= (CreateAndFailSilent
)op
;
1421 return Op
.create(cafs
.getPath(), cafs
.getData(), zkw
.createACL(cafs
.getPath()),
1422 CreateMode
.PERSISTENT
);
1423 } else if (op
instanceof DeleteNodeFailSilent
) {
1424 DeleteNodeFailSilent dnfs
= (DeleteNodeFailSilent
)op
;
1425 return Op
.delete(dnfs
.getPath(), -1);
1426 } else if (op
instanceof SetData
) {
1427 SetData sd
= (SetData
) op
;
1428 return Op
.setData(sd
.getPath(), sd
.getData(), sd
.getVersion());
1430 throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1431 + op
.getClass().getName());
1435 // Static boolean for warning about useMulti because ideally there will be one warning per
1436 // process instance. It is fine if two threads end up racing on this for a bit. We do not
1437 // need to use an atomic type for this, that is overkill. The goal of reducing the number of
1438 // warnings from many to one or a few at startup is still achieved.
1439 private static boolean useMultiWarn
= true;
1442 * Use ZooKeeper's multi-update functionality.
1444 * If all of the following are true:
1445 * - runSequentialOnMultiFailure is true
1446 * - on calling multi, we get a ZooKeeper exception that can be handled by a sequential call(*)
1448 * - we retry the operations one-by-one (sequentially)
1450 * Note *: an example is receiving a NodeExistsException from a "create" call. Without multi,
1451 * a user could call "createAndFailSilent" to ensure that a node exists if they don't care who
1452 * actually created the node (i.e. the NodeExistsException from ZooKeeper is caught).
1453 * This will cause all operations in the multi to fail, however, because
1454 * the NodeExistsException that zk.create throws will fail the multi transaction.
1455 * In this case, if the previous conditions hold, the commands are run sequentially, which should
1456 * result in the correct final state, but means that the operations will not run atomically.
1458 * @throws KeeperException if a ZooKeeper operation fails
1460 public static void multiOrSequential(ZKWatcher zkw
, List
<ZKUtilOp
> ops
,
1461 boolean runSequentialOnMultiFailure
) throws KeeperException
{
1465 if (useMultiWarn
) { // Only check and warn at first use
1466 if (zkw
.getConfiguration().get("hbase.zookeeper.useMulti") != null) {
1467 LOG
.warn("hbase.zookeeper.useMulti is deprecated. Default to true always.");
1469 useMultiWarn
= false;
1471 List
<Op
> zkOps
= new LinkedList
<>();
1472 for (ZKUtilOp op
: ops
) {
1473 zkOps
.add(toZooKeeperOp(zkw
, op
));
1476 zkw
.getRecoverableZooKeeper().multi(zkOps
);
1477 } catch (KeeperException ke
) {
1478 switch (ke
.code()) {
1484 // if we get an exception that could be solved by running sequentially
1485 // (and the client asked us to), then break out and run sequentially
1486 if (runSequentialOnMultiFailure
) {
1487 LOG
.info("multi exception: {}; running operations sequentially " +
1488 "(runSequentialOnMultiFailure=true); {}", ke
.toString(),
1489 ops
.stream().map(o
-> o
.toString()).collect(Collectors
.joining(",")));
1490 processSequentially(zkw
, ops
);
1496 } catch (InterruptedException ie
) {
1497 zkw
.interruptedException(ie
);
1501 private static void processSequentially(ZKWatcher zkw
, List
<ZKUtilOp
> ops
)
1502 throws KeeperException
, NoNodeException
{
1503 for (ZKUtilOp op
: ops
) {
1504 if (op
instanceof CreateAndFailSilent
) {
1505 createAndFailSilent(zkw
, (CreateAndFailSilent
) op
);
1506 } else if (op
instanceof DeleteNodeFailSilent
) {
1507 deleteNodeFailSilent(zkw
, (DeleteNodeFailSilent
) op
);
1508 } else if (op
instanceof SetData
) {
1509 setData(zkw
, (SetData
) op
);
1511 throw new UnsupportedOperationException("Unexpected ZKUtilOp type: "
1512 + op
.getClass().getName());
1518 // ZooKeeper cluster information
1521 private static void logRetrievedMsg(final ZKWatcher zkw
,
1522 final String znode
, final byte [] data
, final boolean watcherSet
) {
1523 if (!LOG
.isTraceEnabled()) {
1527 LOG
.trace(zkw
.prefix("Retrieved " + ((data
== null)?
0: data
.length
) +
1528 " byte(s) of data from znode " + znode
+
1529 (watcherSet?
" and set watcher; ": "; data=") +
1530 (data
== null?
"null": data
.length
== 0?
"empty": (
1531 zkw
.getZNodePaths().isMetaZNodePath(znode
)?
1532 getServerNameOrEmptyString(data
):
1533 znode
.startsWith(zkw
.getZNodePaths().backupMasterAddressesZNode
)?
1534 getServerNameOrEmptyString(data
):
1535 StringUtils
.abbreviate(Bytes
.toStringBinary(data
), 32)))));
1538 private static String
getServerNameOrEmptyString(final byte [] data
) {
1540 return ProtobufUtil
.parseServerNameFrom(data
).toString();
1541 } catch (DeserializationException e
) {
1547 * Waits for HBase installation's base (parent) znode to become available.
1548 * @throws IOException on ZK errors
1550 public static void waitForBaseZNode(Configuration conf
) throws IOException
{
1551 LOG
.info("Waiting until the base znode is available");
1552 String parentZNode
= conf
.get(HConstants
.ZOOKEEPER_ZNODE_PARENT
,
1553 HConstants
.DEFAULT_ZOOKEEPER_ZNODE_PARENT
);
1554 ZooKeeper zk
= new ZooKeeper(ZKConfig
.getZKQuorumServersString(conf
),
1555 conf
.getInt(HConstants
.ZK_SESSION_TIMEOUT
,
1556 HConstants
.DEFAULT_ZK_SESSION_TIMEOUT
), EmptyWatcher
.instance
);
1558 final int maxTimeMs
= 10000;
1559 final int maxNumAttempts
= maxTimeMs
/ HConstants
.SOCKET_RETRY_WAIT_MS
;
1561 KeeperException keeperEx
= null;
1564 for (int attempt
= 0; attempt
< maxNumAttempts
; ++attempt
) {
1566 if (zk
.exists(parentZNode
, false) != null) {
1567 LOG
.info("Parent znode exists: {}", parentZNode
);
1571 } catch (KeeperException e
) {
1574 Threads
.sleepWithoutInterrupt(HConstants
.SOCKET_RETRY_WAIT_MS
);
1579 } catch (InterruptedException ex
) {
1580 Thread
.currentThread().interrupt();
1583 if (keeperEx
!= null) {
1584 throw new IOException(keeperEx
);
1589 * Convert a {@link DeserializationException} to a more palatable {@link KeeperException}.
1590 * Used when can't let a {@link DeserializationException} out w/o changing public API.
1591 * @param e Exception to convert
1592 * @return Converted exception
1594 public static KeeperException
convert(final DeserializationException e
) {
1595 KeeperException ke
= new KeeperException
.DataInconsistencyException();
1601 * Recursively print the current state of ZK (non-transactional)
1602 * @param root name of the root directory in zk to print
1604 public static void logZKTree(ZKWatcher zkw
, String root
) {
1605 if (!LOG
.isDebugEnabled()) {
1609 LOG
.debug("Current zk system:");
1610 String prefix
= "|-";
1611 LOG
.debug(prefix
+ root
);
1613 logZKTree(zkw
, root
, prefix
);
1614 } catch (KeeperException e
) {
1615 throw new RuntimeException(e
);
1620 * Helper method to print the current state of the ZK tree.
1621 * @see #logZKTree(ZKWatcher, String)
1622 * @throws KeeperException if an unexpected exception occurs
1624 private static void logZKTree(ZKWatcher zkw
, String root
, String prefix
)
1625 throws KeeperException
{
1626 List
<String
> children
= ZKUtil
.listChildrenNoWatch(zkw
, root
);
1628 if (children
== null) {
1632 for (String child
: children
) {
1633 LOG
.debug(prefix
+ child
);
1634 String node
= ZNodePaths
.joinZNode(root
.equals("/") ?
"" : root
, child
);
1635 logZKTree(zkw
, node
, prefix
+ "---");
1640 * @param position the position to serialize
1641 * @return Serialized protobuf of <code>position</code> with pb magic prefix prepended suitable
1642 * for use as content of an wal position in a replication queue.
1644 public static byte[] positionToByteArray(final long position
) {
1645 byte[] bytes
= ReplicationProtos
.ReplicationHLogPosition
.newBuilder().setPosition(position
)
1646 .build().toByteArray();
1647 return ProtobufUtil
.prependPBMagic(bytes
);
1651 * @param bytes - Content of a WAL position znode.
1652 * @return long - The current WAL position.
1653 * @throws DeserializationException if the WAL position cannot be parsed
1655 public static long parseWALPositionFrom(final byte[] bytes
) throws DeserializationException
{
1656 if (bytes
== null) {
1657 throw new DeserializationException("Unable to parse null WAL position.");
1659 if (ProtobufUtil
.isPBMagicPrefix(bytes
)) {
1660 int pblen
= ProtobufUtil
.lengthOfPBMagic();
1661 ReplicationProtos
.ReplicationHLogPosition
.Builder builder
=
1662 ReplicationProtos
.ReplicationHLogPosition
.newBuilder();
1663 ReplicationProtos
.ReplicationHLogPosition position
;
1665 ProtobufUtil
.mergeFrom(builder
, bytes
, pblen
, bytes
.length
- pblen
);
1666 position
= builder
.build();
1667 } catch (IOException e
) {
1668 throw new DeserializationException(e
);
1670 return position
.getPosition();
1672 if (bytes
.length
> 0) {
1673 return Bytes
.toLong(bytes
);