Repository: hbase Updated Branches: refs/heads/branch-2 7517f9326 -> 3be51a85b
HBASE-18441 ZookeeperWatcher#interruptedException should throw exception Project: http://git-wip-us.apache.org/repos/asf/hbase/repo Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/3be51a85 Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/3be51a85 Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/3be51a85 Branch: refs/heads/branch-2 Commit: 3be51a85ba4c1636bdebadef2ce1d1eda53098fa Parents: 7517f93 Author: Yu Li <l...@apache.org> Authored: Wed Jul 26 15:19:11 2017 +0800 Committer: Yu Li <l...@apache.org> Committed: Wed Jul 26 15:21:25 2017 +0800 ---------------------------------------------------------------------- .../hbase/zookeeper/ZooKeeperWatcher.java | 32 ++++++++++++-------- .../hadoop/hbase/zookeeper/TestZKUtil.java | 22 ++++++++++++++ 2 files changed, 41 insertions(+), 13 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hbase/blob/3be51a85/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index f18b8ba..6bec352 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -207,7 +207,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { } catch(KeeperException.NoNodeException nne) { return; } catch(InterruptedException ie) { - interruptedException(ie); + interruptedExceptionNoThrow(ie, false); } catch (IOException|KeeperException e) { LOG.warn("Received exception while checking and setting zookeeper ACLs", e); } @@ -587,20 +587,26 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { /** * Handles InterruptedExceptions in client calls. - * <p> - * This may be temporary but for now this gives one place to deal with these. - * <p> - * TODO: Currently, this method does nothing. - * Is this ever expected to happen? Do we abort or can we let it run? - * Maybe this should be logged as WARN? It shouldn't happen? - * <p> - * @param ie + * @param ie the InterruptedException instance thrown + * @throws KeeperException the exception to throw, transformed from the InterruptedException + */ + public void interruptedException(InterruptedException ie) throws KeeperException { + interruptedExceptionNoThrow(ie, true); + // Throw a system error exception to let upper level handle it + throw new KeeperException.SystemErrorException(); + } + + /** + * Log the InterruptedException and interrupt current thread + * @param ie The IterruptedException to log + * @param throwLater Whether we will throw the exception latter */ - public void interruptedException(InterruptedException ie) { - LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie); - // At least preserver interrupt. + public void interruptedExceptionNoThrow(InterruptedException ie, boolean throwLater) { + LOG.debug(prefix("Received InterruptedException, will interrupt current thread" + + (throwLater ? " and rethrow a SystemErrorException" : "")), + ie); + // At least preserve interrupt. Thread.currentThread().interrupt(); - // no-op } /** http://git-wip-us.apache.org/repos/asf/hbase/blob/3be51a85/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java ---------------------------------------------------------------------- diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java index 0e1ab92..53dcdbc 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java @@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.Perms; import org.apache.zookeeper.data.ACL; @@ -34,6 +35,7 @@ import org.apache.zookeeper.data.Id; import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.mockito.Mockito; /** * @@ -93,4 +95,24 @@ public class TestZKUtil { Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user5")))); Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6")))); } + + public void testInterruptedDuringAction() + throws ZooKeeperConnectionException, IOException, KeeperException, InterruptedException { + final RecoverableZooKeeper recoverableZk = Mockito.mock(RecoverableZooKeeper.class); + ZooKeeperWatcher zkw = new ZooKeeperWatcher(HBaseConfiguration.create(), "unittest", null) { + @Override + public RecoverableZooKeeper getRecoverableZooKeeper() { + return recoverableZk; + } + }; + Mockito.doThrow(new InterruptedException()).when(recoverableZk) + .getChildren(zkw.znodePaths.baseZNode, null); + try { + ZKUtil.listChildrenNoWatch(zkw, zkw.znodePaths.baseZNode); + } catch (KeeperException.SystemErrorException e) { + // expected + return; + } + Assert.fail("Should have thrown KeeperException but not"); + } }