HDFS-10261. TestBookKeeperHACheckpoints doesn't handle ephemeral HTTP ports. Contributed by Eric Badger.

(cherry picked from commit 9ba1e5af06)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperHACheckpoints.java

(cherry picked from commit 6e37c5fe80)
This commit is contained in:
Kihwal Lee 2016-04-05 17:00:42 -05:00
parent 76e0bb7a19
commit dd701c9800
1 changed files with 33 additions and 14 deletions

View File

@ -17,6 +17,8 @@
*/
package org.apache.hadoop.contrib.bkjournal;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
@ -27,6 +29,9 @@ import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import java.net.BindException;
import java.util.Random;
/**
* Runs the same tests as TestStandbyCheckpoints, but
* using a bookkeeper journal manager as the shared directory
@ -35,6 +40,9 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
private static BKJMUtil bkutil = null;
static int numBookies = 3;
static int journalCount = 0;
private final Random random = new Random();
private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
@SuppressWarnings("rawtypes")
@Override
@ -45,23 +53,34 @@ public class TestBookKeeperHACheckpoints extends TestStandbyCheckpoints {
BKJMUtil.createJournalURI("/checkpointing" + journalCount++)
.toString());
BKJMUtil.addJournalManagerDefinition(conf);
MiniDFSNNTopology topology = new MiniDFSNNTopology()
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10001))
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10002)));
cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(topology)
.numDataNodes(1)
.manageNameDfsSharedDirs(false)
.build();
cluster.waitActive();
int retryCount = 0;
while (true) {
try {
int basePort = 10060 + random.nextInt(100) * 2;
MiniDFSNNTopology topology = new MiniDFSNNTopology()
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
fs = HATestUtil.configureFailoverFs(cluster, conf);
cluster = new MiniDFSCluster.Builder(conf)
.nnTopology(topology)
.numDataNodes(1)
.manageNameDfsSharedDirs(false)
.build();
cluster.waitActive();
nn0 = cluster.getNameNode(0);
nn1 = cluster.getNameNode(1);
fs = HATestUtil.configureFailoverFs(cluster, conf);
cluster.transitionToActive(0);
cluster.transitionToActive(0);
++retryCount;
break;
} catch (BindException e) {
LOG.info("Set up MiniDFSCluster failed due to port conflicts, retry "
+ retryCount + " times");
}
}
}
@BeforeClass