HBASE-24211: [Addendum] Create table is slow in large cluster when AccessController is enabled. (#1658)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Mohammad Arshad 2020-05-07 00:32:33 +05:30 committed by GitHub
parent febf15f1ea
commit d1aff27e06
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
2 changed files with 49 additions and 7 deletions

View File

@ -697,7 +697,23 @@ public class ZKUtil {
*/
public static byte [] getDataAndWatch(ZooKeeperWatcher zkw, String znode)
throws KeeperException {
return getDataInternal(zkw, znode, null, true);
return getDataInternal(zkw, znode, null, true, true);
}
/**
* Get the data at the specified znode and set a watch.
* Returns the data and sets a watch if the node exists. Returns null and no
* watch is set if the node does not exist or there is an exception.
*
* @param zkw zk reference
* @param znode path of node
* @param throwOnInterrupt if false then just interrupt the thread, do not throw exception
* @return data of the specified znode, or null
* @throws KeeperException if unexpected zookeeper exception
*/
public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode, boolean throwOnInterrupt)
throws KeeperException {
return getDataInternal(zkw, znode, null, true, throwOnInterrupt);
}
/**
@ -714,11 +730,11 @@ public class ZKUtil {
*/
public static byte[] getDataAndWatch(ZooKeeperWatcher zkw, String znode,
Stat stat) throws KeeperException {
return getDataInternal(zkw, znode, stat, true);
return getDataInternal(zkw, znode, stat, true, true);
}
private static byte[] getDataInternal(ZooKeeperWatcher zkw, String znode, Stat stat,
boolean watcherSet)
boolean watcherSet, boolean throwOnInterrupt)
throws KeeperException {
try {
byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
@ -736,7 +752,11 @@ public class ZKUtil {
return null;
} catch (InterruptedException e) {
LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
zkw.interruptedException(e);
if (throwOnInterrupt) {
zkw.interruptedException(e);
} else {
zkw.interruptedExceptionNoThrow(e, true);
}
return null;
}
}
@ -796,7 +816,29 @@ public class ZKUtil {
* @deprecated Unused
*/
public static List<NodeAndData> getChildDataAndWatchForNewChildren(
ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
ZooKeeperWatcher zkw, String baseNode) throws KeeperException {
return getChildDataAndWatchForNewChildren(zkw, baseNode, true);
}
/**
* Returns the date of child znodes of the specified znode. Also sets a watch on
* the specified znode which will capture a NodeDeleted event on the specified
* znode as well as NodeChildrenChanged if any children of the specified znode
* are created or deleted.
*
* Returns null if the specified node does not exist. Otherwise returns a
* list of children of the specified node. If the node exists but it has no
* children, an empty list will be returned.
*
* @param zkw zk reference
* @param baseNode path of node to list and watch children of
* @param throwOnInterrupt if false then just interrupt the thread, do not throw exception
* @return list of data of children of the specified node, an empty list if the node
* exists but has no children, and null if the node does not exist
* @throws KeeperException if unexpected zookeeper exception
*/
public static List<NodeAndData> getChildDataAndWatchForNewChildren(
ZooKeeperWatcher zkw, String baseNode, boolean throwOnInterrupt) throws KeeperException {
List<String> nodes =
ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
if (nodes != null) {
@ -807,7 +849,7 @@ public class ZKUtil {
return Collections.emptyList();
}
String nodePath = ZKUtil.joinZNode(baseNode, node);
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath, throwOnInterrupt);
newNodes.add(new NodeAndData(nodePath, data));
}
return newNodes;

View File

@ -200,7 +200,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable
@Override public void run() {
try {
final List<ZKUtil.NodeAndData> nodeList =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode, false);
refreshNodes(nodeList);
} catch (KeeperException ke) {