HDFS-6054. MiniQJMHACluster should not use static port to avoid binding failure in unit test. (Yongjun Zhang)

This commit is contained in:
Yongjun Zhang 2016-01-19 22:54:47 -08:00
parent 1acc509b45
commit 57d0a94305
3 changed files with 47 additions and 12 deletions

View File

@ -2599,6 +2599,9 @@ Release 2.8.0 - UNRELEASED
HDFS-9623. Update example configuration of block state change log in HDFS-9623. Update example configuration of block state change log in
log4j.properties. (Masatake Iwasaki via aajisaka) log4j.properties. (Masatake Iwasaki via aajisaka)
HDFS-6054. MiniQJMHACluster should not use static port to avoid binding
failure in unit test. (Yongjun Zhang)
Release 2.7.3 - UNRELEASED Release 2.7.3 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -42,7 +42,6 @@ public class MiniQJMHACluster {
public static final String NAMESERVICE = "ns1"; public static final String NAMESERVICE = "ns1";
private static final Random RANDOM = new Random(); private static final Random RANDOM = new Random();
private int basePort = 10000;
public static class Builder { public static class Builder {
private final Configuration conf; private final Configuration conf;
@ -92,9 +91,12 @@ public static MiniDFSNNTopology createDefaultTopology(int basePort) {
private MiniQJMHACluster(Builder builder) throws IOException { private MiniQJMHACluster(Builder builder) throws IOException {
this.conf = builder.conf; this.conf = builder.conf;
int retryCount = 0; int retryCount = 0;
int basePort = 10000;
while (true) { while (true) {
try { try {
basePort = 10000 + RANDOM.nextInt(1000) * 4; basePort = 10000 + RANDOM.nextInt(1000) * 4;
LOG.info("Set MiniQJMHACluster basePort to " + basePort);
// start 3 journal nodes // start 3 journal nodes
journalCluster = new MiniJournalCluster.Builder(conf).format(true) journalCluster = new MiniJournalCluster.Builder(conf).format(true)
.build(); .build();
@ -104,7 +106,7 @@ private MiniQJMHACluster(Builder builder) throws IOException {
// start cluster with specified NameNodes // start cluster with specified NameNodes
MiniDFSNNTopology topology = createDefaultTopology(builder.numNNs, basePort); MiniDFSNNTopology topology = createDefaultTopology(builder.numNNs, basePort);
initHAConf(journalURI, builder.conf, builder.numNNs); initHAConf(journalURI, builder.conf, builder.numNNs, basePort);
// First start up the NNs just to format the namespace. The MinIDFSCluster // First start up the NNs just to format the namespace. The MinIDFSCluster
// has no way to just format the NameNodes without also starting them. // has no way to just format the NameNodes without also starting them.
@ -123,16 +125,21 @@ private MiniQJMHACluster(Builder builder) throws IOException {
// restart the cluster // restart the cluster
cluster.restartNameNodes(); cluster.restartNameNodes();
++retryCount;
break; break;
} catch (BindException e) { } catch (BindException e) {
if (cluster != null) {
cluster.shutdown(true);
cluster = null;
}
++retryCount;
LOG.info("MiniQJMHACluster port conflicts, retried " + LOG.info("MiniQJMHACluster port conflicts, retried " +
retryCount + " times"); retryCount + " times");
} }
} }
} }
private Configuration initHAConf(URI journalURI, Configuration conf, int numNNs) { private Configuration initHAConf(URI journalURI, Configuration conf,
int numNNs, int basePort) {
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY, conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
journalURI.toString()); journalURI.toString());

View File

@ -27,10 +27,14 @@
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import java.io.IOException; import java.io.IOException;
import java.net.BindException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.LinkedList; import java.util.LinkedList;
import java.util.Random;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
@ -62,10 +66,13 @@
@RunWith(Parameterized.class) @RunWith(Parameterized.class)
public class TestFailureToReadEdits { public class TestFailureToReadEdits {
private static final Log LOG =
LogFactory.getLog(TestFailureToReadEdits.class);
private static final String TEST_DIR1 = "/test1"; private static final String TEST_DIR1 = "/test1";
private static final String TEST_DIR2 = "/test2"; private static final String TEST_DIR2 = "/test2";
private static final String TEST_DIR3 = "/test3"; private static final String TEST_DIR3 = "/test3";
private static final Random RANDOM = new Random();
private final TestType clusterType; private final TestType clusterType;
private Configuration conf; private Configuration conf;
@ -105,12 +112,30 @@ public void setUpCluster() throws Exception {
HAUtil.setAllowStandbyReads(conf, true); HAUtil.setAllowStandbyReads(conf, true);
if (clusterType == TestType.SHARED_DIR_HA) { if (clusterType == TestType.SHARED_DIR_HA) {
MiniDFSNNTopology topology = MiniQJMHACluster.createDefaultTopology(10000); int basePort = 10000;
int retryCount = 0;
while (true) {
try {
basePort = 10000 + RANDOM.nextInt(1000) * 4;
LOG.info("Set SHARED_DIR_HA cluster's basePort to " + basePort);
MiniDFSNNTopology topology =
MiniQJMHACluster.createDefaultTopology(basePort);
cluster = new MiniDFSCluster.Builder(conf) cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(topology) .nnTopology(topology)
.numDataNodes(0) .numDataNodes(0)
.checkExitOnShutdown(false) .checkExitOnShutdown(false)
.build(); .build();
break;
} catch (BindException e) {
if (cluster != null) {
cluster.shutdown(true);
cluster = null;
}
++retryCount;
LOG.info("SHARED_DIR_HA: MiniQJMHACluster port conflicts, retried " +
retryCount + " times " + e);
}
}
} else { } else {
Builder builder = new MiniQJMHACluster.Builder(conf); Builder builder = new MiniQJMHACluster.Builder(conf);
builder.getDfsBuilder().numDataNodes(0).checkExitOnShutdown(false); builder.getDfsBuilder().numDataNodes(0).checkExitOnShutdown(false);