@InterfaceAudience.Private public final class ZKUtil extends Object
Contains only static methods and constants.
 Methods all throw KeeperException if there is an unexpected zookeeper exception, so
 callers of these methods must handle appropriately. If ZK is required for the operation, the
 server will need to be aborted.
| Modifier and Type | Class and Description | 
|---|---|
| static class  | ZKUtil.NodeAndDataDeprecated. 
 Unused | 
| static class  | ZKUtil.ZKUtilOpRepresents an action taken by ZKUtil, e.g. | 
| Modifier and Type | Field and Description | 
|---|---|
| private static org.slf4j.Logger | LOG | 
| private static boolean | useMultiWarn | 
| Modifier | Constructor and Description | 
|---|---|
| private  | ZKUtil() | 
| Modifier and Type | Method and Description | 
|---|---|
| static void | asyncCreate(ZKWatcher zkw,
           String znode,
           byte[] data,
           org.apache.zookeeper.AsyncCallback.StringCallback cb,
           Object ctx)Async creates the specified node with the specified data. | 
| static int | checkExists(ZKWatcher zkw,
           String znode)Check if the specified node exists. | 
| static org.apache.zookeeper.KeeperException | convert(DeserializationException e)Convert a  DeserializationExceptionto a more palatableKeeperException. | 
| static void | createAndFailSilent(ZKWatcher zkw,
                   String znode)Creates the specified node, iff the node does not exist. | 
| static void | createAndFailSilent(ZKWatcher zkw,
                   String znode,
                   byte[] data)Creates the specified node containing specified data, iff the node does not exist. | 
| private static void | createAndFailSilent(ZKWatcher zkw,
                   ZKUtil.ZKUtilOp.CreateAndFailSilent cafs) | 
| static int | createAndWatch(ZKWatcher zkw,
              String znode,
              byte[] data)Creates the specified node with the specified data and watches it. | 
| static boolean | createEphemeralNodeAndWatch(ZKWatcher zkw,
                           String znode,
                           byte[] data)Set the specified znode to be an ephemeral node carrying the specified data. | 
| static boolean | createNodeIfNotExistsAndWatch(ZKWatcher zkw,
                             String znode,
                             byte[] data)Creates the specified znode to be a persistent node carrying the specified data. | 
| static String | createNodeIfNotExistsNoWatch(ZKWatcher zkw,
                            String znode,
                            byte[] data,
                            org.apache.zookeeper.CreateMode createMode)Creates the specified znode with the specified data but does not watch it. | 
| static void | createSetData(ZKWatcher zkw,
             String znode,
             byte[] data)Set data into node creating node if it doesn't yet exist. | 
| static void | createWithParents(ZKWatcher zkw,
                 String znode)Creates the specified node and all parent nodes required for it to exist. | 
| static void | createWithParents(ZKWatcher zkw,
                 String znode,
                 byte[] data)Creates the specified node and all parent nodes required for it to exist. | 
| static void | deleteChildrenRecursively(ZKWatcher zkw,
                         String node)Delete all the children of the specified node but not the node itself. | 
| static void | deleteChildrenRecursivelyMultiOrSequential(ZKWatcher zkw,
                                          boolean runSequentialOnMultiFailure,
                                          String... pathRoots)Delete all the children of the specified node but not the node itself. | 
| static void | deleteNode(ZKWatcher zkw,
          String node)Delete the specified node. | 
| static boolean | deleteNode(ZKWatcher zkw,
          String node,
          int version)Delete the specified node with the specified version. | 
| static void | deleteNodeFailSilent(ZKWatcher zkw,
                    String node)Deletes the specified node. | 
| private static void | deleteNodeFailSilent(ZKWatcher zkw,
                    ZKUtil.ZKUtilOp.DeleteNodeFailSilent dnfs) | 
| static void | deleteNodeRecursively(ZKWatcher zkw,
                     String node)Delete the specified node and all of it's children. | 
| static void | deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw,
                                      boolean runSequentialOnMultiFailure,
                                      String... pathRoots)Delete the specified node and its children. | 
| (package private) static int | estimateSize(ZKUtil.ZKUtilOp op) | 
| static List<ZKUtil.NodeAndData> | getChildDataAndWatchForNewChildren(ZKWatcher zkw,
                                  String baseNode)Deprecated. 
 Unused | 
| static List<ZKUtil.NodeAndData> | getChildDataAndWatchForNewChildren(ZKWatcher zkw,
                                  String baseNode,
                                  boolean throwOnInterrupt)Deprecated. 
 Unused | 
| static byte[] | getData(ZKWatcher zkw,
       String znode)Get znode data. | 
| static byte[] | getDataAndWatch(ZKWatcher zkw,
               String znode)Get the data at the specified znode and set a watch. | 
| static byte[] | getDataAndWatch(ZKWatcher zkw,
               String znode,
               boolean throwOnInterrupt)Get the data at the specified znode and set a watch. | 
| static byte[] | getDataAndWatch(ZKWatcher zkw,
               String znode,
               org.apache.zookeeper.data.Stat stat)Get the data at the specified znode and set a watch. | 
| private static byte[] | getDataInternal(ZKWatcher zkw,
               String znode,
               org.apache.zookeeper.data.Stat stat,
               boolean watcherSet,
               boolean throwOnInterrupt) | 
| static byte[] | getDataNoWatch(ZKWatcher zkw,
              String znode,
              org.apache.zookeeper.data.Stat stat)Get the data at the specified znode without setting a watch. | 
| static String | getNodeName(String path)Get the name of the current node from the specified fully-qualified path. | 
| static int | getNumberOfChildren(ZKWatcher zkw,
                   String znode)Get the number of children of the specified node. | 
| static String | getParent(String node)Returns the full path of the immediate parent of the specified node. | 
| private static String | getServerNameOrEmptyString(byte[] data) | 
| static List<String> | listChildrenAndWatchForNewChildren(ZKWatcher zkw,
                                  String znode)Lists the children znodes of the specified znode. | 
| static List<String> | listChildrenAndWatchThem(ZKWatcher zkw,
                        String znode)List all the children of the specified znode, setting a watch for children changes and also
 setting a watch on every individual child in order to get the NodeCreated and NodeDeleted
 events. | 
| private static List<String> | listChildrenBFSAndWatchThem(ZKWatcher zkw,
                           String znode)BFS Traversal of all the children under path, with the entries in the list, in the same order
 as that of the traversal. | 
| private static List<String> | listChildrenBFSNoWatch(ZKWatcher zkw,
                      String znode)BFS Traversal of all the children under path, with the entries in the list, in the same order
 as that of the traversal. | 
| static List<String> | listChildrenNoWatch(ZKWatcher zkw,
                   String znode)Lists the children of the specified znode without setting any watches. | 
| private static void | logRetrievedMsg(ZKWatcher zkw,
               String znode,
               byte[] data,
               boolean watcherSet) | 
| static void | logZKTree(ZKWatcher zkw,
         String root)Recursively print the current state of ZK (non-transactional) | 
| private static void | logZKTree(ZKWatcher zkw,
         String root,
         String prefix)Helper method to print the current state of the ZK tree. | 
| static void | multiOrSequential(ZKWatcher zkw,
                 List<ZKUtil.ZKUtilOp> ops,
                 boolean runSequentialOnMultiFailure)Use ZooKeeper's multi-update functionality. | 
| static boolean | nodeHasChildren(ZKWatcher zkw,
               String znode)Checks if the specified znode has any children. | 
| static long | parseWALPositionFrom(byte[] bytes) | 
| (package private) static List<List<ZKUtil.ZKUtilOp>> | partitionOps(List<ZKUtil.ZKUtilOp> ops,
            int maxPartitionSize)Partition the list of  opsby size (usingestimateSize(ZKUtilOp)). | 
| static byte[] | positionToByteArray(long position) | 
| private static void | processSequentially(ZKWatcher zkw,
                   List<ZKUtil.ZKUtilOp> ops) | 
| static void | setData(ZKWatcher zkw,
       String znode,
       byte[] data)Sets the data of the existing znode to be the specified data. | 
| static boolean | setData(ZKWatcher zkw,
       String znode,
       byte[] data,
       int expectedVersion)Sets the data of the existing znode to be the specified data. | 
| private static void | setData(ZKWatcher zkw,
       ZKUtil.ZKUtilOp.SetData setData) | 
| static boolean | setWatchIfNodeExists(ZKWatcher zkw,
                    String znode)Watch the specified znode, but only if exists. | 
| private static void | submitBatchedMultiOrSequential(ZKWatcher zkw,
                              boolean runSequentialOnMultiFailure,
                              List<ZKUtil.ZKUtilOp> ops)Chunks the provided  opswhen their approximate size exceeds the the configured limit. | 
| private static org.apache.zookeeper.Op | toZooKeeperOp(ZKWatcher zkw,
             ZKUtil.ZKUtilOp op)Convert from ZKUtilOp to ZKOp | 
| static void | updateExistingNodeData(ZKWatcher zkw,
                      String znode,
                      byte[] data,
                      int expectedVersion)Deprecated. 
 Unused | 
| static void | waitForBaseZNode(org.apache.hadoop.conf.Configuration conf)Waits for HBase installation's base (parent) znode to become available. | 
| static boolean | watchAndCheckExists(ZKWatcher zkw,
                   String znode)Watch the specified znode for delete/create/change events. | 
private static final org.slf4j.Logger LOG
private static boolean useMultiWarn
private ZKUtil()
public static String getParent(String node)
node - path to get parent ofpublic static String getNodeName(String path)
path - fully-qualified pathpublic static boolean watchAndCheckExists(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to watchorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static boolean setWatchIfNodeExists(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to watchorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static int checkExists(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to watchorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static List<String> listChildrenAndWatchForNewChildren(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to list and watch children oforg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static List<String> listChildrenAndWatchThem(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zookeeper referenceznode - node to get children of and watchorg.apache.zookeeper.KeeperException - if a ZooKeeper operation failspublic static List<String> listChildrenNoWatch(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zookeeper referenceznode - node to get childrenorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static boolean nodeHasChildren(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to check for children oforg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static int getNumberOfChildren(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of node to count children oforg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static byte[] getData(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException, InterruptedException
org.apache.zookeeper.KeeperExceptionInterruptedExceptionpublic static byte[] getDataAndWatch(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static byte[] getDataAndWatch(ZKWatcher zkw, String znode, boolean throwOnInterrupt) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodethrowOnInterrupt - if false then just interrupt the thread, do not throw exceptionorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static byte[] getDataAndWatch(ZKWatcher zkw, String znode, org.apache.zookeeper.data.Stat stat) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodestat - object to populate the version of the znodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionprivate static byte[] getDataInternal(ZKWatcher zkw, String znode, org.apache.zookeeper.data.Stat stat, boolean watcherSet, boolean throwOnInterrupt) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic static byte[] getDataNoWatch(ZKWatcher zkw, String znode, org.apache.zookeeper.data.Stat stat) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodestat - node status to get if node existsorg.apache.zookeeper.KeeperException - if unexpected zookeeper exception@Deprecated public static List<ZKUtil.NodeAndData> getChildDataAndWatchForNewChildren(ZKWatcher zkw, String baseNode) throws org.apache.zookeeper.KeeperException
zkw - zk referencebaseNode - path of node to list and watch children oforg.apache.zookeeper.KeeperException - if unexpected zookeeper exception@Deprecated public static List<ZKUtil.NodeAndData> getChildDataAndWatchForNewChildren(ZKWatcher zkw, String baseNode, boolean throwOnInterrupt) throws org.apache.zookeeper.KeeperException
zkw - zk referencebaseNode - path of node to list and watch children ofthrowOnInterrupt - if true then just interrupt the thread, do not throw exceptionorg.apache.zookeeper.KeeperException - if unexpected zookeeper exception@Deprecated public static void updateExistingNodeData(ZKWatcher zkw, String znode, byte[] data, int expectedVersion) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - the path to the ZNodedata - the data to store in ZooKeeperexpectedVersion - the expected versionorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionKeeperException.BadVersionException - if version mismatchpublic static boolean setData(ZKWatcher zkw, String znode, byte[] data, int expectedVersion) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
 If the node does not exist, a KeeperException.NoNodeException will be thrown.
 
If their is a version mismatch, method returns null.
No watches are set but setting data will trigger other watchers of this node.
If there is another problem, a KeeperException will be thrown.
zkw - zk referenceznode - path of nodedata - data to set for nodeexpectedVersion - version expected when setting dataorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionorg.apache.zookeeper.KeeperException.NoNodeExceptionpublic static void createSetData(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodedata - data to set for nodeorg.apache.zookeeper.KeeperException - if a ZooKeeper operation failspublic static void setData(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
 If the node does not exist, a KeeperException.NoNodeException will be thrown.
 
No watches are set but setting data will trigger other watchers of this node.
If there is another problem, a KeeperException will be thrown.
zkw - zk referenceznode - path of nodedata - data to set for nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionorg.apache.zookeeper.KeeperException.NoNodeExceptionprivate static void setData(ZKWatcher zkw, ZKUtil.ZKUtilOp.SetData setData) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
org.apache.zookeeper.KeeperExceptionorg.apache.zookeeper.KeeperException.NoNodeExceptionpublic static boolean createEphemeralNodeAndWatch(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodedata - data of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static boolean createNodeIfNotExistsAndWatch(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodedata - data of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static String createNodeIfNotExistsNoWatch(ZKWatcher zkw, String znode, byte[] data, org.apache.zookeeper.CreateMode createMode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodedata - data of nodecreateMode - specifying whether the node to be created is ephemeral and/or sequentialorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static int createAndWatch(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NodeExistsException
Throws an exception if the node already exists.
The node created is persistent and open access.
Returns the version number of the created node if successful.
zkw - zk referenceznode - path of node to createdata - data of node to createorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionKeeperException.NodeExistsException - if node already existspublic static void asyncCreate(ZKWatcher zkw, String znode, byte[] data, org.apache.zookeeper.AsyncCallback.StringCallback cb, Object ctx)
Throws an exception if the node already exists.
The node created is persistent and open access.
zkw - zk referenceznode - path of node to createdata - data of node to createcb - the callback to use for the creationctx - the context to use for the creationpublic static void createAndFailSilent(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static void createAndFailSilent(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodedata - a byte array data to store in the znodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionprivate static void createAndFailSilent(ZKWatcher zkw, ZKUtil.ZKUtilOp.CreateAndFailSilent cafs) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic static void createWithParents(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static void createWithParents(ZKWatcher zkw, String znode, byte[] data) throws org.apache.zookeeper.KeeperException
zkw - zk referenceznode - path of nodeorg.apache.zookeeper.KeeperException - if unexpected zookeeper exceptionpublic static void deleteNode(ZKWatcher zkw, String node) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic static boolean deleteNode(ZKWatcher zkw, String node, int version) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic static void deleteNodeFailSilent(ZKWatcher zkw, String node) throws org.apache.zookeeper.KeeperException
zkw - reference to the ZKWatcher which also contains configuration and operationnode - the node to deleteorg.apache.zookeeper.KeeperException - if a ZooKeeper operation failsprivate static void deleteNodeFailSilent(ZKWatcher zkw, ZKUtil.ZKUtilOp.DeleteNodeFailSilent dnfs) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionpublic static void deleteNodeRecursively(ZKWatcher zkw, String node) throws org.apache.zookeeper.KeeperException
If the node does not exist, just returns.
Sets no watches. Throws all exceptions besides dealing with deletion of children.
org.apache.zookeeper.KeeperExceptionpublic static void deleteChildrenRecursively(ZKWatcher zkw, String node) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException - if a ZooKeeper operation failspublic static void deleteChildrenRecursivelyMultiOrSequential(ZKWatcher zkw, boolean runSequentialOnMultiFailure, String... pathRoots) throws org.apache.zookeeper.KeeperException
Sets no watches. Throws all exceptions besides dealing with deletion of children.
If the following is true:
KeeperException.NotEmptyException - if node has children while deleting n * if unexpected
                                           ZooKeeper exception n * if an invalid path is
                                           specifiedorg.apache.zookeeper.KeeperExceptionpublic static void deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw, boolean runSequentialOnMultiFailure, String... pathRoots) throws org.apache.zookeeper.KeeperException
Sets no watches. Throws all exceptions besides dealing with deletion of children.
If the following is true:
KeeperException.NotEmptyException - if node has children while deleting n * if unexpected
                                           ZooKeeper exception n * if an invalid path is
                                           specifiedorg.apache.zookeeper.KeeperExceptionprivate static void submitBatchedMultiOrSequential(ZKWatcher zkw, boolean runSequentialOnMultiFailure, List<ZKUtil.ZKUtilOp> ops) throws org.apache.zookeeper.KeeperException
ops when their approximate size exceeds the the configured limit.
 Take caution that this can ONLY be used for operations where atomicity is not important, e.g.
 deletions. It must not be used when atomicity of the operations is critical.zkw - reference to the ZKWatcher which contains
                                    configuration and constantsrunSequentialOnMultiFailure - if true when we get a ZooKeeper exception that could retry
                                    the operations one-by-one (sequentially)ops - list of ZKUtilOp ZKUtil.ZKUtilOp to partition while
                                    submitting batched multi or sequentialorg.apache.zookeeper.KeeperException - unexpected ZooKeeper Exception / Zookeeper unreachablestatic List<List<ZKUtil.ZKUtilOp>> partitionOps(List<ZKUtil.ZKUtilOp> ops, int maxPartitionSize)
ops by size (using estimateSize(ZKUtilOp)).static int estimateSize(ZKUtil.ZKUtilOp op)
private static List<String> listChildrenBFSNoWatch(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionprivate static List<String> listChildrenBFSAndWatchThem(ZKWatcher zkw, String znode) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperExceptionprivate static org.apache.zookeeper.Op toZooKeeperOp(ZKWatcher zkw, ZKUtil.ZKUtilOp op) throws UnsupportedOperationException
UnsupportedOperationExceptionpublic static void multiOrSequential(ZKWatcher zkw, List<ZKUtil.ZKUtilOp> ops, boolean runSequentialOnMultiFailure) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException - if a ZooKeeper operation failsprivate static void processSequentially(ZKWatcher zkw, List<ZKUtil.ZKUtilOp> ops) throws org.apache.zookeeper.KeeperException, org.apache.zookeeper.KeeperException.NoNodeException
org.apache.zookeeper.KeeperExceptionorg.apache.zookeeper.KeeperException.NoNodeExceptionprivate static void logRetrievedMsg(ZKWatcher zkw, String znode, byte[] data, boolean watcherSet)
private static String getServerNameOrEmptyString(byte[] data)
public static void waitForBaseZNode(org.apache.hadoop.conf.Configuration conf) throws IOException
IOException - on ZK errorspublic static org.apache.zookeeper.KeeperException convert(DeserializationException e)
DeserializationException to a more palatable KeeperException. Used
 when can't let a DeserializationException out w/o changing public API.e - Exception to convertpublic static void logZKTree(ZKWatcher zkw, String root)
root - name of the root directory in zk to printprivate static void logZKTree(ZKWatcher zkw, String root, String prefix) throws org.apache.zookeeper.KeeperException
org.apache.zookeeper.KeeperException - if an unexpected exception occurslogZKTree(ZKWatcher, String)public static byte[] positionToByteArray(long position)
position - the position to serializeposition with pb magic prefix prepended suitable
         for use as content of an wal position in a replication queue.public static long parseWALPositionFrom(byte[] bytes) throws DeserializationException
bytes - - Content of a WAL position znode.DeserializationException - if the WAL position cannot be parsedCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.