HBASE-24327 : Flaky connection in TestMasterShutdown#testMasterShutdo… (#1690)

Signed-off-by: Bharath Vissapragada <bharathv@apache.org>
This commit is contained in:
Viraj Jasani 2020-05-13 02:38:47 +05:30 committed by GitHub
parent dc360f0f41
commit a782531633
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 46 additions and 28 deletions

View File

@ -22,6 +22,8 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics;
@ -31,7 +33,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -151,7 +152,9 @@ public class TestMasterShutdown {
hbaseCluster = new LocalHBaseCluster(htu.getConfiguration(), options.getNumMasters(),
options.getNumRegionServers(), options.getMasterClass(), options.getRsClass());
final MasterThread masterThread = hbaseCluster.getMasters().get(0);
masterThread.start();
final CompletableFuture<Void> shutdownFuture = CompletableFuture.runAsync(() -> {
// Switching to master registry exacerbated a race in the master bootstrap that can result
// in a lost shutdown command (HBASE-8422, HBASE-23836). The race is essentially because
// the server manager in HMaster is not initialized by the time shutdown() RPC (below) is
@ -160,10 +163,35 @@ public class TestMasterShutdown {
// manager is usually init'ed in time for the RPC to be made. For now, adding an explicit
// wait() in the test, waiting for the server manager to become available.
final long timeout = TimeUnit.MINUTES.toMillis(10);
assertNotEquals("Timeout waiting for server manager to become available.",
-1, Waiter.waitFor(htu.getConfiguration(), timeout,
() -> masterThread.getMaster().getServerManager() != null));
assertNotEquals("timeout waiting for server manager to become available.", -1,
htu.waitFor(timeout, () -> masterThread.getMaster().getServerManager() != null));
// Master has come up far enough that we can terminate it without creating a zombie.
LOG.debug("Attempting to establish connection.");
try {
// HBASE-24327 : (Resolve Flaky connection issues)
// shutdown() RPC can have flaky ZK connection issues.
// e.g
// ERROR [RpcServer.priority.RWQ.Fifo.read.handler=1,queue=1,port=53033]
// master.HMaster(2878): ZooKeeper exception trying to set cluster as down in ZK
// org.apache.zookeeper.KeeperException$SystemErrorException:
// KeeperErrorCode = SystemError
//
// However, even when above flakes happen, shutdown call does get completed even if
// RPC call has failure. Hence, subsequent retries will never succeed as HMaster is
// already shutdown. Hence, it can fail. To resolve it, after making one shutdown()
// call, we are ignoring IOException.
htu.getConnection().getAdmin().shutdown();
LOG.info("Shutdown RPC sent.");
} catch (IOException | CompletionException e) {
LOG.warn("Failed to establish connection.", e);
} catch (Throwable e) {
LOG.warn("Something unexpected happened.", e);
throw new RuntimeException(e);
}
});
shutdownFuture.join();
masterThread.join();
} finally {
if (hbaseCluster != null) {
@ -186,19 +214,9 @@ public class TestMasterShutdown {
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
// don't need a long write pipeline for this test.
conf.setInt("dfs.replication", 1);
return conf;
}
/**
* Create a new {@link Configuration} based on {@code baseConf} that has ZooKeeper connection
* settings tuned very aggressively. The resulting client is used within a retry loop, so there's
* no value in having the client itself do the retries. We want to iterate on the base
* configuration because we're waiting for the mini-cluster to start and set it's ZK client port.
*
* @return a new, configured {@link Configuration} instance.
*/
private static Configuration createResponsiveZkConfig(final Configuration baseConf) {
final Configuration conf = HBaseConfiguration.create(baseConf);
// reduce client retries
conf.setInt("hbase.client.retries.number", 3);
// Recoverable ZK configs are tuned more aggressively
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3);
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100);
return conf;