HBASE-24327 : Flaky connection in TestMasterShutdown (ADDENDUM)
This commit is contained in:
parent
a73132c62b
commit
15627bb722
|
@ -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,7 +154,6 @@ 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
|
||||||
|
@ -167,7 +166,6 @@ public class TestMasterShutdown {
|
||||||
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.
|
||||||
|
@ -182,16 +180,14 @@ public class TestMasterShutdown {
|
||||||
// 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();
|
||||||
LOG.info("Shutdown RPC sent.");
|
} catch (RetriesExhaustedException e) {
|
||||||
} catch (IOException | CompletionException e) {
|
if (e.getCause() instanceof ConnectionClosedException) {
|
||||||
LOG.warn("Failed to establish connection.", e);
|
LOG.info("Connection is Closed to the cluster. The cluster is already down.", e);
|
||||||
} catch (Throwable e) {
|
} else {
|
||||||
LOG.warn("Something unexpected happened.", e);
|
throw 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);
|
||||||
|
|
Loading…
Reference in New Issue