diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java index 308ef410780..f21e8772c15 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/ZKPermissionWatcher.java @@ -38,6 +38,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; /** @@ -120,7 +121,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable public void nodeCreated(String path) { waitUntilStarted(); if (path.equals(aclZNode)) { - executor.submit(new Runnable() { + asyncProcessNodeUpdate(new Runnable() { @Override public void run() { try { @@ -141,7 +142,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable public void nodeDeleted(final String path) { waitUntilStarted(); if (aclZNode.equals(ZKUtil.getParent(path))) { - executor.submit(new Runnable() { + asyncProcessNodeUpdate(new Runnable() { @Override public void run() { String table = ZKUtil.getNodeName(path); @@ -159,7 +160,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable public void nodeDataChanged(final String path) { waitUntilStarted(); if (aclZNode.equals(ZKUtil.getParent(path))) { - executor.submit(new Runnable() { + asyncProcessNodeUpdate(new Runnable() { @Override public void run() { // update cache on an existing table node @@ -198,7 +199,7 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable LOG.error("Error reading data from zookeeper for path "+path, ke); watcher.abort("ZooKeeper error get node children for path "+path, ke); } - executor.submit(new Runnable() { + asyncProcessNodeUpdate(new Runnable() { // allows subsequent nodeChildrenChanged event to preempt current processing of // nodeChildrenChanged event @Override @@ -211,6 +212,20 @@ public class ZKPermissionWatcher extends ZooKeeperListener implements Closeable } } + private void asyncProcessNodeUpdate(Runnable runnable) { + if (!executor.isShutdown()) { + try { + executor.submit(runnable); + } catch (RejectedExecutionException e) { + if (executor.isShutdown()) { + LOG.warn("aclZNode changed after ZKPermissionWatcher was shutdown"); + } else { + throw e; + } + } + } + } + private void refreshNodes(List nodes, AtomicReference ref) { for (ZKUtil.NodeAndData n : nodes) { if (ref != null && ref.get() != null) {