HBASE-18441 ZookeeperWatcher#interruptedException should throw exception

This commit is contained in:
Yu Li 2017-07-26 15:19:11 +08:00
parent 7517f93265
commit 3be51a85ba
2 changed files with 41 additions and 13 deletions

View File

@ -207,7 +207,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
} catch(KeeperException.NoNodeException nne) { } catch(KeeperException.NoNodeException nne) {
return; return;
} catch(InterruptedException ie) { } catch(InterruptedException ie) {
interruptedException(ie); interruptedExceptionNoThrow(ie, false);
} catch (IOException|KeeperException e) { } catch (IOException|KeeperException e) {
LOG.warn("Received exception while checking and setting zookeeper ACLs", 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. * Handles InterruptedExceptions in client calls.
* <p> * @param ie the InterruptedException instance thrown
* This may be temporary but for now this gives one place to deal with these. * @throws KeeperException the exception to throw, transformed from the InterruptedException
* <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
*/ */
public void interruptedException(InterruptedException ie) { public void interruptedException(InterruptedException ie) throws KeeperException {
LOG.debug(prefix("Received InterruptedException, doing nothing here"), ie); interruptedExceptionNoThrow(ie, true);
// At least preserver interrupt. // 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 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(); Thread.currentThread().interrupt();
// no-op
} }
/** /**

View File

@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.Superusers;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooDefs.Ids; import org.apache.zookeeper.ZooDefs.Ids;
import org.apache.zookeeper.ZooDefs.Perms; import org.apache.zookeeper.ZooDefs.Perms;
import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.ACL;
@ -34,6 +35,7 @@ import org.apache.zookeeper.data.Id;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
import org.junit.experimental.categories.Category; 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", "user5"))));
Assert.assertTrue(aclList.contains(new ACL(Perms.ALL, new Id("sasl", "user6")))); 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");
}
} }