HBASE-19753 Miscellany of fixes for hbase-zookeeper tests to make them more robust

First, we add test resources to CLASSPATH when tests run. W/o it, there
was no logging of hbase-zookeeper test output (not sure why I have to
add this here and not over in hbase-server; research turns up nothing
so far).

M hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKMainServer.java
 Improve fail log message.

M hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestReadOnlyZKClient.java
M hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKNodeTracker.java
 Wait until ZK is connected before progressing. On my slow zk, it could
 be a while post construction before zk connected. Using an unconnected
 zk caused test to fail.

M hbase-zookeeper/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKMainServer.java
 Change session timeout to default 30s from 1s which was way too short.

M hbase-zookeeper/src/test/resources/log4j.properties
 Set zk logs to DEBUG level in this module at least.

Adds a ZooKeeperHelper class that has utility to help interacting w/ ZK.
This commit is contained in:
Michael Stack 2018-01-10 14:25:39 -08:00
parent 200c3d3b8e
commit a7f9668c33
8 changed files with 103 additions and 26 deletions

View File

@ -31,6 +31,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.Code; import org.apache.zookeeper.KeeperException.Code;
@ -249,8 +250,8 @@ public final class ReadOnlyZKClient implements Closeable {
@Override @Override
public void exec(ZooKeeper zk) { public void exec(ZooKeeper zk) {
zk.getData(path, false, (rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), zk.getData(path, false,
null); (rc, path, ctx, data, stat) -> onComplete(zk, rc, data, true), null);
} }
}); });
return future; return future;
@ -284,8 +285,17 @@ public final class ReadOnlyZKClient implements Closeable {
private ZooKeeper getZk() throws IOException { private ZooKeeper getZk() throws IOException {
// may be closed when session expired // may be closed when session expired
if (zookeeper == null || !zookeeper.getState().isAlive()) { if (zookeeper == null || !zookeeper.getState().isAlive()) {
zookeeper = new ZooKeeper(connectString, sessionTimeoutMs, e -> { zookeeper = new ZooKeeper(connectString, sessionTimeoutMs, e -> {});
}); int timeout = 10000;
try {
// Before returning, try and ensure we are connected. Don't wait long in case
// we are trying to connect to a cluster that is down. If we fail to connect,
// just catch the exception and carry-on. The first usage will fail and we'll
// cleanup.
zookeeper = ZooKeeperHelper.ensureConnectedZooKeeper(zookeeper, timeout);
} catch (ZooKeeperConnectionException e) {
LOG.warn("Failed connecting after waiting " + timeout + "ms; " + zookeeper);
}
} }
return zookeeper; return zookeeper;
} }

View File

@ -0,0 +1,71 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.ZooKeeper;
/**
* Methods that help working with ZooKeeper
*/
@InterfaceAudience.Private
public final class ZooKeeperHelper {
// This class cannot be instantiated
private ZooKeeperHelper() {
}
/**
* Get a ZooKeeper instance and wait until it connected before returning.
* @param sessionTimeoutMs Used as session timeout passed to the created ZooKeeper AND as the
* timeout to wait on connection establishment.
*/
public static ZooKeeper getConnectedZooKeeper(String connectString, int sessionTimeoutMs)
throws IOException {
ZooKeeper zookeeper = new ZooKeeper(connectString, sessionTimeoutMs, e -> {});
return ensureConnectedZooKeeper(zookeeper, sessionTimeoutMs);
}
/**
* Ensure passed zookeeper is connected.
* @param timeout Time to wait on established Connection
*/
public static ZooKeeper ensureConnectedZooKeeper(ZooKeeper zookeeper, int timeout)
throws ZooKeeperConnectionException {
if (zookeeper.getState().isConnected()) {
return zookeeper;
}
Stopwatch stopWatch = Stopwatch.createStarted();
// Make sure we are connected before we hand it back.
while(!zookeeper.getState().isConnected()) {
Threads.sleep(1);
if (stopWatch.elapsed(TimeUnit.MILLISECONDS) > timeout) {
throw new ZooKeeperConnectionException("Failed connect after waiting " +
stopWatch.elapsed(TimeUnit.MILLISECONDS) + "ms (zk session timeout); " +
zookeeper);
}
}
return zookeeper;
}
}

View File

@ -96,6 +96,9 @@
<plugin> <plugin>
<artifactId>maven-surefire-plugin</artifactId> <artifactId>maven-surefire-plugin</artifactId>
<configuration> <configuration>
<additionalClasspathElements>
<additionalClasspathElement>src/test/resources</additionalClasspathElement>
</additionalClasspathElements>
<properties> <properties>
<property> <property>
<name>listener</name> <name>listener</name>

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.zookeeper; package org.apache.hadoop.hbase.zookeeper;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
@ -28,7 +27,6 @@ import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.ZooKeeperMain; import org.apache.zookeeper.ZooKeeperMain;
import org.apache.hbase.thirdparty.com.google.common.base.Stopwatch;
/** /**
* Tool for running ZookeeperMain from HBase by reading a ZooKeeper server * Tool for running ZookeeperMain from HBase by reading a ZooKeeper server
@ -52,15 +50,9 @@ public class ZKMainServer {
super(args); super(args);
// Make sure we are connected before we proceed. Can take a while on some systems. If we // Make sure we are connected before we proceed. Can take a while on some systems. If we
// run the command without being connected, we get ConnectionLoss KeeperErrorConnection... // run the command without being connected, we get ConnectionLoss KeeperErrorConnection...
Stopwatch stopWatch = Stopwatch.createStarted(); // Make it 30seconds. We dont' have a config in this context and zk doesn't have
while (!this.zk.getState().isConnected()) { // a timeout until after connection. 30000ms is default for zk.
Thread.sleep(1); ZooKeeperHelper.ensureConnectedZooKeeper(this.zk, 30000);
if (stopWatch.elapsed(TimeUnit.SECONDS) > 10) {
throw new InterruptedException("Failed connect after waiting " +
stopWatch.elapsed(TimeUnit.SECONDS) + "seconds; state=" + this.zk.getState() +
"; " + this.zk);
}
}
} }
/** /**

View File

@ -1,4 +1,4 @@
/** /*
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -67,8 +67,8 @@ public class TestReadOnlyZKClient {
public static void setUp() throws Exception { public static void setUp() throws Exception {
PORT = UTIL.startMiniZKCluster().getClientPort(); PORT = UTIL.startMiniZKCluster().getClientPort();
ZooKeeper zk = new ZooKeeper("localhost:" + PORT, 10000, e -> { ZooKeeper zk = ZooKeeperHelper.
}); getConnectedZooKeeper("localhost:" + PORT, 10000);
DATA = new byte[10]; DATA = new byte[10];
ThreadLocalRandom.current().nextBytes(DATA); ThreadLocalRandom.current().nextBytes(DATA);
zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zk.create(PATH, DATA, ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
@ -137,8 +137,8 @@ public class TestReadOnlyZKClient {
UTIL.getZkCluster().getZooKeeperServers().get(0).closeSession(sessionId); UTIL.getZkCluster().getZooKeeperServers().get(0).closeSession(sessionId);
// should not reach keep alive so still the same instance // should not reach keep alive so still the same instance
assertSame(zk, RO_ZK.getZooKeeper()); assertSame(zk, RO_ZK.getZooKeeper());
byte [] got = RO_ZK.get(PATH).get();
assertArrayEquals(DATA, RO_ZK.get(PATH).get()); assertArrayEquals(DATA, got);
assertNotNull(RO_ZK.getZooKeeper()); assertNotNull(RO_ZK.getZooKeeper());
assertNotSame(zk, RO_ZK.getZooKeeper()); assertNotSame(zk, RO_ZK.getZooKeeper());
assertNotEquals(sessionId, RO_ZK.getZooKeeper().getSessionId()); assertNotEquals(sessionId, RO_ZK.getZooKeeper().getSessionId());

View File

@ -70,7 +70,8 @@ public class TestZKMainServer {
public void testCommandLineWorks() throws Exception { public void testCommandLineWorks() throws Exception {
System.setSecurityManager(new NoExitSecurityManager()); System.setSecurityManager(new NoExitSecurityManager());
HBaseZKTestingUtility htu = new HBaseZKTestingUtility(); HBaseZKTestingUtility htu = new HBaseZKTestingUtility();
htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 1000); // Make it long so for sure succeeds.
htu.getConfiguration().setInt(HConstants.ZK_SESSION_TIMEOUT, 30000);
htu.startMiniZKCluster(); htu.startMiniZKCluster();
try { try {
ZKWatcher zkw = htu.getZooKeeperWatcher(); ZKWatcher zkw = htu.getZooKeeperWatcher();

View File

@ -129,9 +129,9 @@ public class TestZKNodeTracker {
// Create a completely separate zk connection for test triggers and avoid // Create a completely separate zk connection for test triggers and avoid
// any weird watcher interactions from the test // any weird watcher interactions from the test
final ZooKeeper zkconn = final ZooKeeper zkconn = ZooKeeperHelper.
new ZooKeeper(ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()), 60000, e -> { getConnectedZooKeeper(ZKConfig.getZKQuorumServersString(TEST_UTIL.getConfiguration()),
}); 60000);
// Add the node with data one // Add the node with data one
zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); zkconn.create(node, dataOne, Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);

View File

@ -55,7 +55,7 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %C{2}(%L):
#log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG
log4j.logger.org.apache.hadoop=WARN log4j.logger.org.apache.hadoop=WARN
log4j.logger.org.apache.zookeeper=ERROR log4j.logger.org.apache.zookeeper=DEBUG
log4j.logger.org.apache.hadoop.hbase=DEBUG log4j.logger.org.apache.hadoop.hbase=DEBUG
#These settings are workarounds against spurious logs from the minicluster. #These settings are workarounds against spurious logs from the minicluster.