HADOOP-10980. TestActiveStandbyElector fails occasionally in trunk. Contributed by Eric Badger

(cherry picked from commit c82745432a)

Conflicts:

	hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
This commit is contained in:
Jason Lowe 2016-08-03 20:27:15 +00:00
parent 47e1eabf79
commit 2bf4e76370
3 changed files with 28 additions and 6 deletions

View File

@ -28,6 +28,9 @@ Release 2.7.4 - UNRELEASED
HADOOP-11149. Increase the timeout of TestZKFailoverController. HADOOP-11149. Increase the timeout of TestZKFailoverController.
(Steve Loughran via wheat9) (Steve Loughran via wheat9)
HADOOP-10980. TestActiveStandbyElector fails occasionally in trunk
(Eric Badger via jlowe)
Release 2.7.3 - UNRELEASED Release 2.7.3 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -613,13 +613,13 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
/** /**
* Get a new zookeeper client instance. protected so that test class can * Get a new zookeeper client instance. protected so that test class can
* inherit and pass in a mock object for zookeeper * inherit and mock out the zookeeper instance
* *
* @return new zookeeper client instance * @return new zookeeper client instance
* @throws IOException * @throws IOException
* @throws KeeperException zookeeper connectionloss exception * @throws KeeperException zookeeper connectionloss exception
*/ */
protected synchronized ZooKeeper getNewZooKeeper() throws IOException, protected synchronized ZooKeeper connectToZooKeeper() throws IOException,
KeeperException { KeeperException {
// Unfortunately, the ZooKeeper constructor connects to ZooKeeper and // Unfortunately, the ZooKeeper constructor connects to ZooKeeper and
@ -628,7 +628,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
// we construct the watcher first, and have it block any events it receives // we construct the watcher first, and have it block any events it receives
// before we can set its ZooKeeper reference. // before we can set its ZooKeeper reference.
watcher = new WatcherWithClientRef(); watcher = new WatcherWithClientRef();
ZooKeeper zk = new ZooKeeper(zkHostPort, zkSessionTimeout, watcher); ZooKeeper zk = createZooKeeper();
watcher.setZooKeeperRef(zk); watcher.setZooKeeperRef(zk);
// Wait for the asynchronous success/failure. This may throw an exception // Wait for the asynchronous success/failure. This may throw an exception
@ -641,6 +641,17 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
return zk; return zk;
} }
/**
* Get a new zookeeper client instance. protected so that test class can
* inherit and pass in a mock object for zookeeper
*
* @return new zookeeper client instance
* @throws IOException
*/
protected ZooKeeper createZooKeeper() throws IOException {
return new ZooKeeper(zkHostPort, zkSessionTimeout, watcher);
}
private void fatalError(String errorMessage) { private void fatalError(String errorMessage) {
LOG.fatal(errorMessage); LOG.fatal(errorMessage);
reset(); reset();
@ -772,7 +783,7 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
zkClient = null; zkClient = null;
watcher = null; watcher = null;
} }
zkClient = getNewZooKeeper(); zkClient = connectToZooKeeper();
LOG.debug("Created new connection for " + this); LOG.debug("Created new connection for " + this);
} }

View File

@ -66,7 +66,7 @@ public class TestActiveStandbyElector {
} }
@Override @Override
public ZooKeeper getNewZooKeeper() { public ZooKeeper connectToZooKeeper() {
++count; ++count;
return mockZK; return mockZK;
} }
@ -749,7 +749,15 @@ public class TestActiveStandbyElector {
try { try {
new ActiveStandbyElector("127.0.0.1", 2000, ZK_PARENT_NAME, new ActiveStandbyElector("127.0.0.1", 2000, ZK_PARENT_NAME,
Ids.OPEN_ACL_UNSAFE, Collections.<ZKAuthInfo> emptyList(), mockApp, Ids.OPEN_ACL_UNSAFE, Collections.<ZKAuthInfo> emptyList(), mockApp,
CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_DEFAULT); CommonConfigurationKeys.HA_FC_ELECTOR_ZK_OP_RETRIES_DEFAULT) {
@Override
protected ZooKeeper createZooKeeper() throws IOException {
return Mockito.mock(ZooKeeper.class);
}
};
Assert.fail("Did not throw zookeeper connection loss exceptions!"); Assert.fail("Did not throw zookeeper connection loss exceptions!");
} catch (KeeperException ke) { } catch (KeeperException ke) {
GenericTestUtils.assertExceptionContains( "ConnectionLoss", ke); GenericTestUtils.assertExceptionContains( "ConnectionLoss", ke);