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:
parent
7f4683bafa
commit
edd8829cfc
|
@ -185,25 +185,28 @@ public class ZKPermissionWatcher extends ZKListener implements Closeable {
|
|||
public void nodeChildrenChanged(final String path) {
|
||||
waitUntilStarted();
|
||||
if (path.equals(aclZNode)) {
|
||||
try {
|
||||
final List<ZKUtil.NodeAndData> nodeList =
|
||||
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
|
||||
// preempt any existing nodeChildrenChanged event processing
|
||||
if (childrenChangedFuture != null && !childrenChangedFuture.isDone()) {
|
||||
boolean cancelled = childrenChangedFuture.cancel(true);
|
||||
if (!cancelled) {
|
||||
// task may have finished between our check and attempted cancel, this is fine.
|
||||
if (! childrenChangedFuture.isDone()) {
|
||||
LOG.warn("Could not cancel processing node children changed event, " +
|
||||
"please file a JIRA and attach logs if possible.");
|
||||
if (!childrenChangedFuture.isDone()) {
|
||||
LOG.warn("Could not cancel processing node children changed event, "
|
||||
+ "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) {
|
||||
LOG.error("Error reading data from zookeeper for path "+path, ke);
|
||||
watcher.abort("ZooKeeper error get node children for path "+path, ke);
|
||||
String msg = "ZooKeeper error while reading node children data for path " + path;
|
||||
LOG.error(msg, ke);
|
||||
watcher.abort(msg, ke);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -634,9 +634,24 @@ public final class ZKUtil {
|
|||
* @return data of the specified znode, or null
|
||||
* @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 {
|
||||
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,
|
||||
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,
|
||||
boolean watcherSet)
|
||||
private static byte[] getDataInternal(ZKWatcher zkw, String znode, Stat stat, boolean watcherSet,
|
||||
boolean throwOnInterrupt)
|
||||
throws KeeperException {
|
||||
try {
|
||||
byte [] data = zkw.getRecoverableZooKeeper().getData(znode, zkw, stat);
|
||||
|
@ -675,7 +690,11 @@ public final class ZKUtil {
|
|||
return null;
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn(zkw.prefix("Unable to get data of znode " + znode), e);
|
||||
if (throwOnInterrupt) {
|
||||
zkw.interruptedException(e);
|
||||
} else {
|
||||
zkw.interruptedExceptionNoThrow(e, true);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -735,15 +754,43 @@ public final class ZKUtil {
|
|||
* @deprecated Unused
|
||||
*/
|
||||
@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(
|
||||
ZKWatcher zkw, String baseNode) throws KeeperException {
|
||||
ZKWatcher zkw, String baseNode, boolean throwOnInterrupt) throws KeeperException {
|
||||
List<String> nodes =
|
||||
ZKUtil.listChildrenAndWatchForNewChildren(zkw, baseNode);
|
||||
if (nodes != null) {
|
||||
List<NodeAndData> newNodes = new ArrayList<>();
|
||||
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);
|
||||
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath);
|
||||
byte[] data = ZKUtil.getDataAndWatch(zkw, nodePath, throwOnInterrupt);
|
||||
newNodes.add(new NodeAndData(nodePath, data));
|
||||
}
|
||||
return newNodes;
|
||||
|
|
Loading…
Reference in New Issue