HBASE-19794 TestZooKeeper hangs

Kill backup master first

Add some cleanup around NamespaceManager

Shorten the timeout waiting on namespace manager as workaround
until we have better soln for interrupting ongoing client rpcs.
Do it in general for all tests.

Signed-off-by: zhangduo <zhangduo@apache.org>
This commit is contained in:
Michael Stack 2018-01-20 20:11:10 +08:00 committed by zhangduo
parent 7fe4aa6fe4
commit 27d00f5861
7 changed files with 72 additions and 23 deletions

View File

@ -1802,4 +1802,10 @@ possible configurations would overwhelm and obscure the important.
Number of rows in a batch operation above which a warning will be logged.
</description>
</property>
<property>
<name>hbase.master.wait.on.service.seconds</name>
<value>30</value>
<description>Default is 5 minutes. Make it 30 seconds for tests. See
HBASE-19794 for some context.</description>
</property>
</configuration>

View File

@ -76,6 +76,10 @@ public class ZKNamespaceManager extends ZKListener {
}
}
public void stop() throws IOException {
this.watcher.unregisterListener(this);
}
public NamespaceDescriptor get(String name) {
return cache.get(name);
}

View File

@ -69,6 +69,10 @@ class ClusterSchemaServiceImpl extends AbstractService implements ClusterSchemaS
protected void doStop() {
// This is no stop for the table manager.
notifyStopped();
TableNamespaceManager tnsm = getTableNamespaceManager();
if (tnsm != null) {
tnsm.stop("Stopping");
}
}
@Override

View File

@ -2692,9 +2692,11 @@ public class HMaster extends HRegionServer implements MasterServices {
@Override
public void stop(String msg) {
super.stop(msg);
if (this.activeMasterManager != null) {
this.activeMasterManager.stop();
if (!isStopped()) {
super.stop(msg);
if (this.activeMasterManager != null) {
this.activeMasterManager.stop();
}
}
}

View File

@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.Stoppable;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZKNamespaceManager;
import org.apache.hadoop.hbase.client.Delete;
@ -68,8 +69,9 @@ import org.slf4j.LoggerFactory;
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC",
justification="TODO: synchronize access on nsTable but it is done in tiers above and this " +
"class is going away/shrinking")
public class TableNamespaceManager {
public class TableNamespaceManager implements Stoppable {
private static final Logger LOG = LoggerFactory.getLogger(TableNamespaceManager.class);
private volatile boolean stopped = false;
private Configuration conf;
private MasterServices masterServices;
@ -368,4 +370,27 @@ public class TableNamespaceManager {
}
return maxRegions;
}
@Override
public boolean isStopped() {
return this.stopped;
}
@Override
public void stop(String why) {
if (this.stopped) {
return;
}
try {
this.zkNamespaceManager.stop();
} catch (IOException ioe) {
LOG.warn("Failed NamespaceManager close", ioe);
}
try {
this.nsTable.close();
} catch (IOException ioe) {
LOG.warn("Failed Namespace Table close", ioe);
}
this.stopped = true;
}
}

View File

@ -837,12 +837,22 @@ public class MiniHBaseCluster extends HBaseCluster {
* impossible to bring the mini-cluster back for clean shutdown.
*/
public void killAll() {
// Do backups first.
MasterThread activeMaster = null;
for (MasterThread masterThread : getMasterThreads()) {
if (!masterThread.getMaster().isActiveMaster()) {
masterThread.getMaster().abort("killAll");
} else {
activeMaster = masterThread;
}
}
// Do active after.
if (activeMaster != null) {
activeMaster.getMaster().abort("killAll");
}
for (RegionServerThread rst : getRegionServerThreads()) {
rst.getRegionServer().abort("killAll");
}
for (MasterThread masterThread : getMasterThreads()) {
masterThread.getMaster().abort("killAll", new Throwable());
}
}
@Override

View File

@ -27,10 +27,8 @@ import java.io.IOException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.IntStream;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
@ -104,7 +102,7 @@ public class TestZKAsyncRegistry {
String expectedClusterId = TEST_UTIL.getHBaseCluster().getMaster().getClusterId();
assertEquals("Expected " + expectedClusterId + ", found=" + clusterId,
expectedClusterId, clusterId);
assertEquals(TEST_UTIL.getHBaseCluster().getClusterStatus().getServersSize(),
assertEquals(TEST_UTIL.getHBaseCluster().getClusterMetrics().getLiveServerMetrics().size(),
REGISTRY.getCurrentNrHRS().get().intValue());
assertEquals(TEST_UTIL.getHBaseCluster().getMaster().getServerName(),
REGISTRY.getMasterAddress().get());
@ -122,19 +120,19 @@ public class TestZKAsyncRegistry {
@Test
public void testIndependentZKConnections() throws IOException {
ReadOnlyZKClient zk1 = REGISTRY.getZKClient();
Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
try (ZKAsyncRegistry otherRegistry = new ZKAsyncRegistry(otherConf)) {
ReadOnlyZKClient zk2 = otherRegistry.getZKClient();
assertNotSame("Using a different configuration / quorum should result in different backing " +
"zk connection.",
zk1, zk2);
assertNotEquals(
"Using a different configrution / quorum should be reflected in the " + "zk connection.",
zk1.getConnectString(), zk2.getConnectString());
try (ReadOnlyZKClient zk1 = REGISTRY.getZKClient()) {
Configuration otherConf = new Configuration(TEST_UTIL.getConfiguration());
otherConf.set(HConstants.ZOOKEEPER_QUORUM, "127.0.0.1");
try (ZKAsyncRegistry otherRegistry = new ZKAsyncRegistry(otherConf)) {
ReadOnlyZKClient zk2 = otherRegistry.getZKClient();
assertNotSame("Using a different configuration / quorum should result in different " +
"backing zk connection.", zk1, zk2);
assertNotEquals(
"Using a different configrution / quorum should be reflected in the zk connection.",
zk1.getConnectString(), zk2.getConnectString());
}
} finally {
LOG.info("DONE!");
}
}
}