HBASE-24211: Create table is slow in large cluster when AccessController is enabled. (#1631)

Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
Mohammad Arshad 2020-05-06 15:12:20 +05:30 committed by Viraj Jasani
parent 7f4683bafa
commit edd8829cfc
No known key found for this signature in database
GPG Key ID: 3AE697641452FC5D
2 changed files with 74 additions and 24 deletions

View File

@ -185,25 +185,28 @@ public class ZKPermissionWatcher extends ZKListener implements Closeable {
public void nodeChildrenChanged(final String path) { public void nodeChildrenChanged(final String path) {
waitUntilStarted(); waitUntilStarted();
if (path.equals(aclZNode)) { if (path.equals(aclZNode)) {
try {
final List<ZKUtil.NodeAndData> nodeList =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
// preempt any existing nodeChildrenChanged event processing // preempt any existing nodeChildrenChanged event processing
if (childrenChangedFuture != null && !childrenChangedFuture.isDone()) { if (childrenChangedFuture != null && !childrenChangedFuture.isDone()) {
boolean cancelled = childrenChangedFuture.cancel(true); boolean cancelled = childrenChangedFuture.cancel(true);
if (!cancelled) { if (!cancelled) {
// task may have finished between our check and attempted cancel, this is fine. // task may have finished between our check and attempted cancel, this is fine.
if (! childrenChangedFuture.isDone()) { if (!childrenChangedFuture.isDone()) {
LOG.warn("Could not cancel processing node children changed event, " + LOG.warn("Could not cancel processing node children changed event, "
"please file a JIRA and attach logs if possible."); + "please file a JIRA and attach logs if possible.");
} }
} }
} }
childrenChangedFuture = asyncProcessNodeUpdate(() -> refreshNodes(nodeList)); childrenChangedFuture = asyncProcessNodeUpdate(() -> {
try {
final List<ZKUtil.NodeAndData> nodeList =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode, false);
refreshNodes(nodeList);
} catch (KeeperException ke) { } catch (KeeperException ke) {
LOG.error("Error reading data from zookeeper for path "+path, ke); String msg = "ZooKeeper error while reading node children data for path " + path;
watcher.abort("ZooKeeper error get node children for path "+path, ke); LOG.error(msg, ke);
watcher.abort(msg, ke);
} }
});
} }
} }

View File

@ -634,9 +634,24 @@ public final class ZKUtil {
* @return data of the specified znode, or null * @return data of the specified znode, or null
* @throws KeeperException if unexpected zookeeper exception * @throws KeeperException if unexpected zookeeper exception
*/ */
public static byte [] getDataAndWatch(ZKWatcher zkw, String znode) public static byte[] getDataAndWatch(ZKWatcher zkw, String znode) throws KeeperException {
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(ZKWatcher zkw, String znode, boolean throwOnInterrupt)
throws KeeperException { throws KeeperException {
return getDataInternal(zkw, znode, null, true); return getDataInternal(zkw, znode, null, true, throwOnInterrupt);
} }
/** /**
@ -653,11 +668,11 @@ public final class ZKUtil {
*/ */
public static byte[] getDataAndWatch(ZKWatcher zkw, String znode, public static byte[] getDataAndWatch(ZKWatcher zkw, String znode,
Stat stat) throws KeeperException { Stat stat) throws KeeperException {
return getDataInternal(zkw, znode, stat, true); return getDataInternal(zkw, znode, stat, true, true);
} }
private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat, private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat, boolean watcherSet,
boolean watcherSet) boolean throwOnInterrupt)
throws KeeperException { throws KeeperException {
try { try {
byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat); byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
@ -675,7 +690,11 @@ public final class ZKUtil {
return null; return null;
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e); LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
if (throwOnInterrupt) {
zkw.interruptedException(e); zkw.interruptedException(e);
} else {
zkw.interruptedExceptionNoThrow(e, true);
}
return null; return null;
} }
} }
@ -735,15 +754,43 @@ public final class ZKUtil {
* @deprecated Unused * @deprecated Unused
*/ */
@Deprecated @Deprecated
public static List<NodeAndData> getChildDataAndWatchForNewChildren(ZKWatcher 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 true 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
* @deprecated Unused
*/
@Deprecated
public static List<NodeAndData> getChildDataAndWatchForNewChildren( public static List<NodeAndData> getChildDataAndWatchForNewChildren(
ZKWatcher zkw, String baseNode) throws KeeperException { ZKWatcher zkw, String baseNode, boolean throwOnInterrupt) throws KeeperException {
List<String> nodes = List<String> nodes =
ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode); ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
if (nodes != null) { if (nodes != null) {
List<NodeAndData> newNodes = new ArrayList<>(); List<NodeAndData> newNodes = new ArrayList<>();
for (String node : nodes) { for (String node : nodes) {
if (Thread.interrupted()) {
// Partial data should not be processed. Cancel processing by sending empty list.
return Collections.emptyList();
}
String nodePath = ZNodePaths.joinZNode(baseNode, node); String nodePath = ZNodePaths.joinZNode(baseNode, node);
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath); byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath, throwOnInterrupt);
newNodes.add(new NodeAndData(nodePath, data)); newNodes.add(new NodeAndData(nodePath, data));
} }
return newNodes; return newNodes;