HBASE-24327 : Flaky connection in TestMasterShutdown (ADDENDUM)

This commit is contained in:
Viraj Jasani 2020-05-16 12:26:51 +05:30
parent a73132c62b
commit 15627bb722
No known key found for this signature in database
GPG Key ID: 3AE697641452FC5D
1 changed files with 35 additions and 39 deletions

View File

@ -22,8 +22,6 @@ import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics; import org.apache.hadoop.hbase.ClusterMetrics;
@ -33,6 +31,8 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.LocalHBaseCluster; import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.StartMiniClusterOption; import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
import org.apache.hadoop.hbase.exceptions.ConnectionClosedException;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread; import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
@ -154,44 +154,40 @@ public class TestMasterShutdown {
final MasterThread masterThread = hbaseCluster.getMasters().get(0); final MasterThread masterThread = hbaseCluster.getMasters().get(0);
masterThread.start(); masterThread.start();
final CompletableFuture<Void> shutdownFuture = CompletableFuture.runAsync(() -> { // Switching to master registry exacerbated a race in the master bootstrap that can result
// 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
// 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
// the server manager in HMaster is not initialized by the time shutdown() RPC (below) is // made to the master. The suspected reason as to why it was uncommon before HBASE-18095
// made to the master. The suspected reason as to why it was uncommon before HBASE-18095 // is because the connection creation with ZK registry is so slow that by then the server
// is because the connection creation with ZK registry is so slow that by then the server // manager is usually init'ed in time for the RPC to be made. For now, adding an explicit
// 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.
// wait() in the test, waiting for the server manager to become available. final long timeout = TimeUnit.MINUTES.toMillis(10);
final long timeout = TimeUnit.MINUTES.toMillis(10); assertNotEquals("timeout waiting for server manager to become available.", -1,
assertNotEquals("timeout waiting for server manager to become available.", -1, htu.waitFor(timeout, () -> masterThread.getMaster().getServerManager() != null));
htu.waitFor(timeout, () -> masterThread.getMaster().getServerManager() != null));
// Master has come up far enough that we can terminate it without creating a zombie. // Master has come up far enough that we can terminate it without creating a zombie.
LOG.debug("Attempting to establish connection."); try {
try { // HBASE-24327 : (Resolve Flaky connection issues)
// HBASE-24327 : (Resolve Flaky connection issues) // shutdown() RPC can have flaky ZK connection issues.
// shutdown() RPC can have flaky ZK connection issues. // e.g
// e.g // ERROR [RpcServer.priority.RWQ.Fifo.read.handler=1,queue=1,port=53033]
// 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
// master.HMaster(2878): ZooKeeper exception trying to set cluster as down in ZK // org.apache.zookeeper.KeeperException$SystemErrorException:
// org.apache.zookeeper.KeeperException$SystemErrorException: // KeeperErrorCode = SystemError
// KeeperErrorCode = SystemError //
// // However, even when above flakes happen, shutdown call does get completed even if
// 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
// 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()
// already shutdown. Hence, it can fail. To resolve it, after making one shutdown() // call, we are ignoring IOException.
// call, we are ignoring IOException. htu.getConnection().getAdmin().shutdown();
htu.getConnection().getAdmin().shutdown(); } catch (RetriesExhaustedException e) {
LOG.info("Shutdown RPC sent."); if (e.getCause() instanceof ConnectionClosedException) {
} catch (IOException | CompletionException e) { LOG.info("Connection is Closed to the cluster. The cluster is already down.", e);
LOG.warn("Failed to establish connection.", e); } else {
} catch (Throwable e) { throw e;
LOG.warn("Something unexpected happened.", e);
throw new RuntimeException(e);
} }
}); }
LOG.info("Shutdown RPC sent.");
shutdownFuture.join();
masterThread.join(); masterThread.join();
} finally { } finally {
if (hbaseCluster != null) { if (hbaseCluster != null) {
@ -215,7 +211,7 @@ public class TestMasterShutdown {
// don't need a long write pipeline for this test. // don't need a long write pipeline for this test.
conf.setInt("dfs.replication", 1); conf.setInt("dfs.replication", 1);
// reduce client retries // reduce client retries
conf.setInt("hbase.client.retries.number", 3); conf.setInt("hbase.client.retries.number", 1);
// Recoverable ZK configs are tuned more aggressively // Recoverable ZK configs are tuned more aggressively
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3); conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY, 3);
conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100); conf.setInt(ReadOnlyZKClient.RECOVERY_RETRY_INTERVAL_MILLIS, 100);