diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 3ef248d..ea28b81 100644 --- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -27,17 +27,20 @@ import java.io.PrintWriter; import java.net.InetSocketAddress; import java.net.Socket; import java.nio.charset.StandardCharsets; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; import java.util.Deque; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; +import org.apache.commons.collections4.IterableUtils; +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.AuthUtil; @@ -71,6 +74,7 @@ import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.DeleteRequest; import org.apache.zookeeper.proto.SetDataRequest; import org.apache.zookeeper.server.ZooKeeperSaslServer; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -122,14 +126,14 @@ public final class ZKUtil { public static RecoverableZooKeeper connect(Configuration conf, String ensemble, Watcher watcher, final String identifier) throws IOException { - if(ensemble == null) { + if (ensemble == null) { throw new IOException("Unable to determine ZooKeeper ensemble"); } int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT); - if (LOG.isTraceEnabled()) { - LOG.trace(identifier + " opening connection to ZooKeeper ensemble=" + ensemble); - } + + LOG.trace("{} opening connection to ZooKeeper ensemble={}", identifier, ensemble); + int retry = conf.getInt("zookeeper.recovery.retry", 3); int retryIntervalMillis = conf.getInt("zookeeper.recovery.retry.intervalmill", 1000); @@ -215,7 +219,7 @@ public final class ZKUtil { // No keytab specified, no auth String keytabFilename = conf.get(keytabFileKey); if (keytabFilename == null) { - LOG.warn("no keytab specified for: " + keytabFileKey); + LOG.warn("no keytab specified for: {}", keytabFileKey); return; } @@ -329,7 +333,7 @@ public final class ZKUtil { * @return name of the current node */ public static String getNodeName(String path) { - return path.substring(path.lastIndexOf("/")+1); + return path.substring(path.lastIndexOf('/') + 1); } // @@ -352,17 +356,17 @@ public final class ZKUtil { Stat s = zkw.getRecoverableZooKeeper().exists(znode, zkw); boolean exists = s != null ? true : false; if (exists) { - LOG.debug(zkw.prefix("Set watcher on existing znode=" + znode)); + LOG.debug("{} Set watcher on existing znode: {}", zkw, znode); } else { - LOG.debug(zkw.prefix("Set watcher on znode that does not yet exist, " + znode)); + LOG.debug("{} Set watcher on znode that does not yet exist: {}", zkw, znode); } return exists; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + LOG.warn("{} Unable to set watcher on znode: {}", zkw, znode, e); zkw.keeperException(e); return false; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + LOG.warn("{} Unable to set watcher on znode: {}", zkw, znode, e); zkw.interruptedException(e); return false; } @@ -386,7 +390,7 @@ public final class ZKUtil { } catch (NoNodeException e) { return false; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode " + znode), e); + LOG.warn("{} Unable to set watcher on znode: {}", zkw, znode, e); zkw.interruptedException(e); return false; } @@ -406,11 +410,11 @@ public final class ZKUtil { Stat s = zkw.getRecoverableZooKeeper().exists(znode, null); return s != null ? s.getVersion() : -1; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); + LOG.warn("{} Unable to set watcher on znode: {}", zkw, znode, e); zkw.keeperException(e); return -1; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to set watcher on znode (" + znode + ")"), e); + LOG.warn("{} Unable to set watcher on znode: {}", zkw, znode, e); zkw.interruptedException(e); return -1; } @@ -443,15 +447,15 @@ public final class ZKUtil { List children = zkw.getRecoverableZooKeeper().getChildren(znode, zkw); return children; } catch(KeeperException.NoNodeException ke) { - LOG.debug(zkw.prefix("Unable to list children of znode " + znode + " " + - "because node does not exist (not an error)")); + LOG.debug("{} Unable to list children of znode {} "+ + "because node does not exist (not an error)", zkw, znode); return null; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); + LOG.warn("{} Unable to list children of znode: {}", zkw, znode, e); zkw.keeperException(e); return null; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode + " "), e); + LOG.warn("{} Unable to list children of znode: {}", zkw, znode, e); zkw.interruptedException(e); return null; } @@ -463,16 +467,13 @@ public final class ZKUtil { * the NodeCreated and NodeDeleted events. * @param zkw zookeeper reference * @param znode node to get children of and watch - * @return list of znode names, null if the node doesn't exist + * @return list of znode names, null if the node does not exist * @throws KeeperException if a ZooKeeper operation fails */ public static List listChildrenAndWatchThem(ZKWatcher zkw, String znode) throws KeeperException { List children = listChildrenAndWatchForNewChildren(zkw, znode); - if (children == null) { - return null; - } - for (String child : children) { + for (String child : IterableUtils.emptyIfNull(children)) { watchAndCheckExists(zkw, ZNodePaths.joinZNode(znode, child)); } return children; @@ -554,15 +555,15 @@ public final class ZKUtil { try { return !zkw.getRecoverableZooKeeper().getChildren(znode, null).isEmpty(); } catch(KeeperException.NoNodeException ke) { - LOG.debug(zkw.prefix("Unable to list children of znode " + znode + - " because node does not exist (not an error)")); + LOG.debug("{} Unable to list children of znode {} " + + "because node does not exist (not an error)", zkw, znode); return false; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); + LOG.warn("{} Unable to list children of znode: {}", zkw, znode, e); zkw.keeperException(e); return false; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to list children of znode " + znode), e); + LOG.warn("{} Unable to list children of znode: {}", zkw, znode, e); zkw.interruptedException(e); return false; } @@ -586,10 +587,10 @@ public final class ZKUtil { try { Stat stat = zkw.getRecoverableZooKeeper().exists(znode, null); return stat == null ? 0 : stat.getNumChildren(); - } catch(KeeperException e) { - LOG.warn(zkw.prefix("Unable to get children of node " + znode)); + } catch (KeeperException e) { + LOG.warn("{} Unable to get children of znode: {}", zkw, znode); zkw.keeperException(e); - } catch(InterruptedException e) { + } catch (InterruptedException e) { zkw.interruptedException(e); } return 0; @@ -611,11 +612,11 @@ public final class ZKUtil { logRetrievedMsg(zkw, znode, data, false); return data; } catch (KeeperException.NoNodeException e) { - LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not an error)")); + LOG.debug("{} Unable to get data of znode {} " + + "because node does not exist (not an error)", zkw, znode); return null; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + LOG.warn("Unable to get data of znode: {}", zkw, znode, e); zkw.keeperException(e); return null; } @@ -664,15 +665,15 @@ public final class ZKUtil { } catch (KeeperException.NoNodeException e) { // This log can get pretty annoying when we cycle on 100ms waits. // Enable trace if you really want to see it. - LOG.trace(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not an error)")); + LOG.trace("{} Unable to get data of znode {} " + + "because node does not exist (not an error)", zkw, znode); return null; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + LOG.warn("{} Unable to get data of znode: {}", zkw, znode, e); zkw.keeperException(e); return null; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + LOG.warn("{} Unable to get data of znode: {}", zkw, znode, e); zkw.interruptedException(e); return null; } @@ -701,15 +702,15 @@ public final class ZKUtil { logRetrievedMsg(zkw, znode, data, false); return data; } catch (KeeperException.NoNodeException e) { - LOG.debug(zkw.prefix("Unable to get data of znode " + znode + " " + - "because node does not exist (not necessarily an error)")); + LOG.debug("{} Unable to get data of znode {} " + + "because node does not exist (not necessarily an error)", zkw, znode); return null; } catch (KeeperException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + LOG.warn("{} Unable to get data of znode: {}", znode, e); zkw.keeperException(e); return null; } catch (InterruptedException e) { - LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); + LOG.warn("{} Unable to get data of znode: {}", zkw, znode, e); zkw.interruptedException(e); return null; } @@ -738,7 +739,7 @@ public final class ZKUtil { List nodes = ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); if (nodes != null) { - List newNodes = new ArrayList<>(); + List newNodes = new ArrayList<>(nodes.size()); for (String node : nodes) { String nodePath = ZNodePaths.joinZNode(baseNode, node); byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath); @@ -811,7 +812,7 @@ public final class ZKUtil { } /** - * Set data into node creating node if it doesn't yet exist. + * Set data into node creating node if it does not yet exist. * Does not set watch. * * @param zkw zk reference @@ -902,7 +903,7 @@ public final class ZKUtil { try { hbaseUser = UserGroupInformation.getCurrentUser().getShortUserName(); } catch (IOException e) { - LOG.debug("Could not acquire current User.", e); + LOG.debug("Could not acquire current User", e); } if (superUsers != null) { List groups = new ArrayList<>(); @@ -911,14 +912,14 @@ public final class ZKUtil { // TODO: Set node ACL for groups when ZK supports this feature groups.add(user); } else { - if(!user.equals(hbaseUser)) { + if (!user.equals(hbaseUser)) { acls.add(new ACL(Perms.ALL, new Id("sasl", user))); } } } if (!groups.isEmpty()) { - LOG.warn("Znode ACL setting for group " + groups - + " is skipped, ZooKeeper doesn't support this feature presently."); + LOG.warn("Znode ACL setting for group {} is skipped, " + + "ZooKeeper does not support this feature presently.", groups); } } // Certain znodes are accessed directly by the client, @@ -971,7 +972,7 @@ public final class ZKUtil { Thread.currentThread().interrupt(); } if(!watchAndCheckExists(zkw, znode)) { - // It did exist but now it doesn't, try again + // It did exist but now it does not, try again return createEphemeralNodeAndWatch(zkw, znode, data); } return ret; @@ -1230,9 +1231,9 @@ public final class ZKUtil { try { zkw.getRecoverableZooKeeper().delete(node, version); return true; - } catch(KeeperException.BadVersionException bve) { + } catch (KeeperException.BadVersionException bve) { return false; - } catch(InterruptedException ie) { + } catch (InterruptedException ie) { zkw.interruptedException(ie); return false; } @@ -1256,8 +1257,8 @@ public final class ZKUtil { DeleteRequest delete = (DeleteRequest)toZooKeeperOp(zkw, dnfs).toRequestRecord(); try { zkw.getRecoverableZooKeeper().delete(delete.getPath(), delete.getVersion()); - } catch(KeeperException.NoNodeException nne) { - } catch(InterruptedException ie) { + } catch (KeeperException.NoNodeException nne) { + } catch (InterruptedException ie) { zkw.interruptedException(ie); } } @@ -1322,7 +1323,7 @@ public final class ZKUtil { public static void deleteChildrenRecursivelyMultiOrSequential( ZKWatcher zkw, boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException { - if (pathRoots == null || pathRoots.length <= 0) { + if (ArrayUtils.isEmpty(pathRoots)) { LOG.warn("Given path is not valid!"); return; } @@ -1334,8 +1335,8 @@ public final class ZKUtil { ops.add(ZKUtilOp.deleteNodeFailSilent(children.get(i))); } } - // atleast one element should exist - if (ops.size() > 0) { + // at least one element should exist + if (!ops.isEmpty()) { multiOrSequential(zkw, ops, runSequentialOnMultiFailure); } } @@ -1372,7 +1373,7 @@ public final class ZKUtil { */ public static void deleteNodeRecursivelyMultiOrSequential(ZKWatcher zkw, boolean runSequentialOnMultiFailure, String... pathRoots) throws KeeperException { - if (pathRoots == null || pathRoots.length <= 0) { + if (ArrayUtils.isEmpty(pathRoots)) { LOG.warn("Given path is not valid!"); return; } @@ -1393,8 +1394,8 @@ public final class ZKUtil { zkw.interruptedException(e); } } - // atleast one element should exist - if (ops.size() > 0) { + // at least one element should exist + if (!ops.isEmpty()) { multiOrSequential(zkw, ops, runSequentialOnMultiFailure); } } @@ -1414,7 +1415,7 @@ public final class ZKUtil { */ private static List listChildrenBFSNoWatch(ZKWatcher zkw, final String znode) throws KeeperException { - Deque queue = new LinkedList<>(); + Deque queue = new ArrayDeque<>(); List tree = new ArrayList<>(); queue.add(znode); while (true) { @@ -1423,10 +1424,7 @@ public final class ZKUtil { break; } List children = listChildrenNoWatch(zkw, node); - if (children == null) { - continue; - } - for (final String child : children) { + for (final String child : IterableUtils.emptyIfNull(children)) { final String childPath = node + "/" + child; queue.add(childPath); tree.add(childPath); @@ -1450,7 +1448,7 @@ public final class ZKUtil { */ private static List listChildrenBFSAndWatchThem(ZKWatcher zkw, final String znode) throws KeeperException { - Deque queue = new LinkedList<>(); + Deque queue = new ArrayDeque<>(); List tree = new ArrayList<>(); queue.add(znode); while (true) { @@ -1459,10 +1457,7 @@ public final class ZKUtil { break; } List children = listChildrenAndWatchThem(zkw, node); - if (children == null) { - continue; - } - for (final String child : children) { + for (final String child : IterableUtils.emptyIfNull(children)) { final String childPath = node + "/" + child; queue.add(childPath); tree.add(childPath); @@ -1549,7 +1544,7 @@ public final class ZKUtil { /** * ZKUtilOp representing deleteNodeFailSilent in ZooKeeper - * (attempt to delete node, ignore error if node doesn't exist) + * (attempt to delete node, ignore error if node does not exist) */ public static final class DeleteNodeFailSilent extends ZKUtilOp { private DeleteNodeFailSilent(String path) { @@ -1662,10 +1657,8 @@ public final class ZKUtil { return; } - List zkOps = new LinkedList<>(); - for (ZKUtilOp op : ops) { - zkOps.add(toZooKeeperOp(zkw, op)); - } + List zkOps = new ArrayList<>(ops.size()); + ops.forEach(op -> zkOps.add(toZooKeeperOp(zkw, op))); try { zkw.getRecoverableZooKeeper().multi(zkOps); } catch (KeeperException ke) { @@ -1677,9 +1670,9 @@ public final class ZKUtil { // if we get an exception that could be solved by running sequentially // (and the client asked us to), then break out and run sequentially if (runSequentialOnMultiFailure) { - LOG.info("On call to ZK.multi, received exception: " + ke.toString() + "." - + " Attempting to run operations sequentially because" - + " runSequentialOnMultiFailure is: " + runSequentialOnMultiFailure + "."); + LOG.info("On call to ZK.multi, received exception: {}." + + " Attempting to run operations sequentially because" + + " runSequentialOnMultiFailure is: {}.", ke, runSequentialOnMultiFailure); processSequentially(zkw, ops); break; } @@ -1830,10 +1823,10 @@ public final class ZKUtil { private static void appendRSZnodes(ZKWatcher zkw, String znode, StringBuilder sb) throws KeeperException { - List stack = new LinkedList<>(); - stack.add(znode); + Deque stack = new ArrayDeque<>(); + stack.push(znode); do { - String znodeToProcess = stack.remove(stack.size() - 1); + String znodeToProcess = stack.pop(); sb.append("\n").append(znodeToProcess).append(": "); byte[] data; try { @@ -1842,7 +1835,7 @@ public final class ZKUtil { zkw.interruptedException(e); return; } - if (data != null && data.length > 0) { // log position + if (ArrayUtils.isNotEmpty(data)) { // log position long position = 0; try { position = ZKUtil.parseWALPositionFrom(ZKUtil.getData(zkw, znodeToProcess)); @@ -1854,9 +1847,9 @@ public final class ZKUtil { } } for (String zNodeChild : ZKUtil.listChildrenNoWatch(zkw, znodeToProcess)) { - stack.add(ZNodePaths.joinZNode(znodeToProcess, zNodeChild)); + stack.push(ZNodePaths.joinZNode(znodeToProcess, zNodeChild)); } - } while (stack.size() > 0); + } while (!stack.isEmpty()); } private static void appendPeersZnodes(ZKWatcher zkw, String peersZnode, @@ -1882,7 +1875,7 @@ public final class ZKUtil { // add the peer-state. appendPeerState(zkw, znodeToProcess, sb); } catch (IOException ipbe) { - LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe); + LOG.warn("Got Exception while parsing peer: {}", znodeToProcess, ipbe); } } } @@ -1907,7 +1900,7 @@ public final class ZKUtil { ProtobufUtil.mergeFrom(builder, peerStateData, pblen, peerStateData.length - pblen); sb.append(builder.getState().name()); } catch (IOException ipbe) { - LOG.warn("Got Exception while parsing peer: " + znodeToProcess, ipbe); + LOG.warn("Got Exception while parsing peer: {}", znodeToProcess, ipbe); } catch (InterruptedException e) { zkw.interruptedException(e); return; @@ -1926,7 +1919,7 @@ public final class ZKUtil { public static String[] getServerStats(String server, int timeout) throws IOException { String[] sp = server.split(":"); - if (sp == null || sp.length == 0) { + if (ArrayUtils.isEmpty(sp)) { return null; } @@ -1945,16 +1938,7 @@ public final class ZKUtil { new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8))) { out.println("stat"); out.flush(); - ArrayList res = new ArrayList<>(); - while (true) { - String line = in.readLine(); - if (line != null) { - res.add(line); - } else { - break; - } - } - return res.toArray(new String[res.size()]); + return IOUtils.readLines(in).toArray(new String[0]); } } } @@ -1965,7 +1949,7 @@ public final class ZKUtil { return; } - LOG.trace(zkw.prefix("Retrieved " + ((data == null)? 0: data.length) + + LOG.trace("{} Retrieved " + ((data == null)? 0: data.length) + " byte(s) of data from znode " + znode + (watcherSet? " and set watcher; ": "; data=") + (data == null? "null": data.length == 0? "empty": ( @@ -1973,14 +1957,14 @@ public final class ZKUtil { getServerNameOrEmptyString(data): znode.startsWith(zkw.znodePaths.backupMasterAddressesZNode)? getServerNameOrEmptyString(data): - StringUtils.abbreviate(Bytes.toStringBinary(data), 32))))); + StringUtils.abbreviate(Bytes.toStringBinary(data), 32))), zkw); } private static String getServerNameOrEmptyString(final byte [] data) { try { return ProtobufUtil.parseServerNameFrom(data).toString(); } catch (DeserializationException e) { - return ""; + return StringUtils.EMPTY; } } @@ -2005,7 +1989,7 @@ public final class ZKUtil { for (int attempt = 0; attempt < maxNumAttempts; ++attempt) { try { if (zk.exists(parentZNode, false) != null) { - LOG.info("Parent znode exists: " + parentZNode); + LOG.info("Parent znode exists: {}", parentZNode); keeperEx = null; break; } @@ -2043,17 +2027,15 @@ public final class ZKUtil { * @param root name of the root directory in zk to print */ public static void logZKTree(ZKWatcher zkw, String root) { - if (!LOG.isDebugEnabled()) { - return; - } - - LOG.debug("Current zk system:"); - String prefix = "|-"; - LOG.debug(prefix + root); - try { - logZKTree(zkw, root, prefix); - } catch (KeeperException e) { - throw new RuntimeException(e); + if (LOG.isDebugEnabled()) { + LOG.debug("Current zk system:"); + final String prefix = "|-"; + LOG.debug(prefix + root); + try { + logZKTree(zkw, root, prefix); + } catch (KeeperException e) { + throw new RuntimeException(e); + } } } @@ -2065,14 +2047,9 @@ public final class ZKUtil { protected static void logZKTree(ZKWatcher zkw, String root, String prefix) throws KeeperException { List children = ZKUtil.listChildrenNoWatch(zkw, root); - - if (children == null) { - return; - } - - for (String child : children) { - LOG.debug(prefix + child); - String node = ZNodePaths.joinZNode(root.equals("/") ? "" : root, child); + for (String child : IterableUtils.emptyIfNull(children)) { + LOG.debug("{}{}", prefix, child); + String node = ZNodePaths.joinZNode("/".equals(root) ? StringUtils.EMPTY : root, child); logZKTree(zkw, node, prefix + "---"); } }