|
@@ -32,6 +32,7 @@ import java.nio.channels.FileChannel;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
|
+import java.util.List;
|
|
|
import java.util.Random;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -41,7 +42,13 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
+import org.apache.hadoop.fs.Path;
|
|
|
+
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.*;
|
|
|
+
|
|
|
+import org.apache.hadoop.ha.HAServiceProtocol;
|
|
|
+import org.apache.hadoop.ha.ServiceFailedException;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSNNTopology.NNConf;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
|
|
|
import org.apache.hadoop.hdfs.protocol.ClientProtocol;
|
|
@@ -65,6 +72,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
|
|
+import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.net.DNSToSwitchMapping;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
@@ -77,6 +85,11 @@ import org.apache.hadoop.tools.GetUserMappingsProtocol;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.ToolRunner;
|
|
|
|
|
|
+import com.google.common.base.Joiner;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.collect.Lists;
|
|
|
+import com.google.common.io.Files;
|
|
|
+
|
|
|
/**
|
|
|
* This class creates a single-process DFS cluster for junit testing.
|
|
|
* The data directories for non-simulated DFS are under the testing directory.
|
|
@@ -102,7 +115,6 @@ public class MiniDFSCluster {
|
|
|
private int nameNodePort = 0;
|
|
|
private int nameNodeHttpPort = 0;
|
|
|
private final Configuration conf;
|
|
|
- private int numNameNodes = 1;
|
|
|
private int numDataNodes = 1;
|
|
|
private boolean format = true;
|
|
|
private boolean manageNameDfsDirs = true;
|
|
@@ -114,21 +126,12 @@ public class MiniDFSCluster {
|
|
|
private String clusterId = null;
|
|
|
private boolean waitSafeMode = true;
|
|
|
private boolean setupHostsFile = false;
|
|
|
- private boolean federation = false;
|
|
|
+ private MiniDFSNNTopology nnTopology = null;
|
|
|
|
|
|
public Builder(Configuration conf) {
|
|
|
this.conf = conf;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * default false - non federated cluster
|
|
|
- * @param val
|
|
|
- * @return Builder object
|
|
|
- */
|
|
|
- public Builder federation (boolean val){
|
|
|
- this.federation = val;
|
|
|
- return this;
|
|
|
- }
|
|
|
/**
|
|
|
* Default: 0
|
|
|
*/
|
|
@@ -145,14 +148,6 @@ public class MiniDFSCluster {
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Default: 1
|
|
|
- */
|
|
|
- public Builder numNameNodes(int val) {
|
|
|
- this.numNameNodes = val;
|
|
|
- return this;
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Default: 1
|
|
|
*/
|
|
@@ -242,6 +237,16 @@ public class MiniDFSCluster {
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Default: a single namenode.
|
|
|
+ * See {@link MiniDFSNNTopology#simpleFederatedTopology(int)} to set up
|
|
|
+ * federated nameservices
|
|
|
+ */
|
|
|
+ public Builder nnTopology(MiniDFSNNTopology topology) {
|
|
|
+ this.nnTopology = topology;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Construct the actual MiniDFSCluster
|
|
|
*/
|
|
@@ -254,15 +259,17 @@ public class MiniDFSCluster {
|
|
|
* Used by builder to create and return an instance of MiniDFSCluster
|
|
|
*/
|
|
|
private MiniDFSCluster(Builder builder) throws IOException {
|
|
|
- LOG.info("starting cluster with " + builder.numNameNodes + " namenodes.");
|
|
|
- nameNodes = new NameNodeInfo[builder.numNameNodes];
|
|
|
- // try to determine if in federation mode
|
|
|
- if(builder.numNameNodes > 1)
|
|
|
- builder.federation = true;
|
|
|
+ if (builder.nnTopology == null) {
|
|
|
+ // If no topology is specified, build a single NN.
|
|
|
+ builder.nnTopology = MiniDFSNNTopology.simpleSingleNN(
|
|
|
+ builder.nameNodePort, builder.nameNodeHttpPort);
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("starting cluster with " +
|
|
|
+ builder.nnTopology.countNameNodes() + " namenodes.");
|
|
|
+ nameNodes = new NameNodeInfo[builder.nnTopology.countNameNodes()];
|
|
|
|
|
|
- initMiniDFSCluster(builder.nameNodePort,
|
|
|
- builder.nameNodeHttpPort,
|
|
|
- builder.conf,
|
|
|
+ initMiniDFSCluster(builder.conf,
|
|
|
builder.numDataNodes,
|
|
|
builder.format,
|
|
|
builder.manageNameDfsDirs,
|
|
@@ -274,7 +281,7 @@ public class MiniDFSCluster {
|
|
|
builder.clusterId,
|
|
|
builder.waitSafeMode,
|
|
|
builder.setupHostsFile,
|
|
|
- builder.federation);
|
|
|
+ builder.nnTopology);
|
|
|
}
|
|
|
|
|
|
public class DataNodeProperties {
|
|
@@ -296,8 +303,8 @@ public class MiniDFSCluster {
|
|
|
new ArrayList<DataNodeProperties>();
|
|
|
private File base_dir;
|
|
|
private File data_dir;
|
|
|
- private boolean federation = false;
|
|
|
private boolean waitSafeMode = true;
|
|
|
+ private boolean federation;
|
|
|
|
|
|
/**
|
|
|
* Stores the information related to a namenode in the cluster
|
|
@@ -488,22 +495,23 @@ public class MiniDFSCluster {
|
|
|
String[] racks, String hosts[],
|
|
|
long[] simulatedCapacities) throws IOException {
|
|
|
this.nameNodes = new NameNodeInfo[1]; // Single namenode in the cluster
|
|
|
- initMiniDFSCluster(nameNodePort, 0, conf, numDataNodes, format,
|
|
|
+ initMiniDFSCluster(conf, numDataNodes, format,
|
|
|
manageNameDfsDirs, manageDataDfsDirs, operation, racks, hosts,
|
|
|
- simulatedCapacities, null, true, false, false);
|
|
|
+ simulatedCapacities, null, true, false,
|
|
|
+ MiniDFSNNTopology.simpleSingleNN(nameNodePort, 0));
|
|
|
}
|
|
|
|
|
|
- private void initMiniDFSCluster(int nameNodePort, int nameNodeHttpPort,
|
|
|
+ private void initMiniDFSCluster(
|
|
|
Configuration conf,
|
|
|
int numDataNodes, boolean format, boolean manageNameDfsDirs,
|
|
|
boolean manageDataDfsDirs, StartupOption operation, String[] racks,
|
|
|
String[] hosts, long[] simulatedCapacities, String clusterId,
|
|
|
- boolean waitSafeMode, boolean setupHostsFile, boolean federation)
|
|
|
+ boolean waitSafeMode, boolean setupHostsFile,
|
|
|
+ MiniDFSNNTopology nnTopology)
|
|
|
throws IOException {
|
|
|
this.conf = conf;
|
|
|
base_dir = new File(determineDfsBaseDir());
|
|
|
data_dir = new File(base_dir, "data");
|
|
|
- this.federation = federation;
|
|
|
this.waitSafeMode = waitSafeMode;
|
|
|
|
|
|
// use alternate RPC engine if spec'd
|
|
@@ -538,28 +546,9 @@ public class MiniDFSCluster {
|
|
|
conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
StaticMapping.class, DNSToSwitchMapping.class);
|
|
|
|
|
|
- Collection<String> nameserviceIds = DFSUtil.getNameServiceIds(conf);
|
|
|
- if(nameserviceIds.size() > 1)
|
|
|
- federation = true;
|
|
|
-
|
|
|
- if (!federation) {
|
|
|
- conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + nameNodePort);
|
|
|
- conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:"
|
|
|
- + nameNodeHttpPort);
|
|
|
- NameNode nn = createNameNode(0, conf, numDataNodes, manageNameDfsDirs,
|
|
|
- format, operation, clusterId);
|
|
|
- nameNodes[0] = new NameNodeInfo(nn, conf);
|
|
|
- FileSystem.setDefaultUri(conf, getURI(0));
|
|
|
- } else {
|
|
|
- if (nameserviceIds.isEmpty()) {
|
|
|
- for (int i = 0; i < nameNodes.length; i++) {
|
|
|
- nameserviceIds.add(NAMESERVICE_ID_PREFIX + i);
|
|
|
- }
|
|
|
- }
|
|
|
- initFederationConf(conf, nameserviceIds, numDataNodes, nameNodePort);
|
|
|
- createFederationNamenodes(conf, nameserviceIds, manageNameDfsDirs, format,
|
|
|
- operation, clusterId);
|
|
|
- }
|
|
|
+ federation = nnTopology.isFederated();
|
|
|
+ createNameNodesAndSetConf(
|
|
|
+ nnTopology, manageNameDfsDirs, format, operation, clusterId, conf);
|
|
|
|
|
|
if (format) {
|
|
|
if (data_dir.exists() && !FileUtil.fullyDelete(data_dir)) {
|
|
@@ -575,51 +564,91 @@ public class MiniDFSCluster {
|
|
|
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
|
|
}
|
|
|
|
|
|
- /** Initialize configuration for federated cluster */
|
|
|
- private static void initFederationConf(Configuration conf,
|
|
|
- Collection<String> nameserviceIds, int numDataNodes, int nnPort) {
|
|
|
- String nameserviceIdList = "";
|
|
|
- for (String nameserviceId : nameserviceIds) {
|
|
|
- // Create comma separated list of nameserviceIds
|
|
|
- if (nameserviceIdList.length() > 0) {
|
|
|
- nameserviceIdList += ",";
|
|
|
- }
|
|
|
- nameserviceIdList += nameserviceId;
|
|
|
- initFederatedNamenodeAddress(conf, nameserviceId, nnPort);
|
|
|
- nnPort = nnPort == 0 ? 0 : nnPort + 2;
|
|
|
+ private void createNameNodesAndSetConf(MiniDFSNNTopology nnTopology,
|
|
|
+ boolean manageNameDfsDirs, boolean format, StartupOption operation,
|
|
|
+ String clusterId,
|
|
|
+ Configuration conf) throws IOException {
|
|
|
+ Preconditions.checkArgument(nnTopology.countNameNodes() > 0,
|
|
|
+ "empty NN topology: no namenodes specified!");
|
|
|
+
|
|
|
+ if (!federation && nnTopology.countNameNodes() == 1) {
|
|
|
+ NNConf onlyNN = nnTopology.getOnlyNameNode();
|
|
|
+ // we only had one NN, set DEFAULT_NAME for it
|
|
|
+ conf.set(FS_DEFAULT_NAME_KEY, "127.0.0.1:" + onlyNN.getIpcPort());
|
|
|
}
|
|
|
- conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIdList);
|
|
|
- }
|
|
|
+
|
|
|
+ int nnCounter = 0;
|
|
|
+ List<String> nsIds = Lists.newArrayList();
|
|
|
+ for (MiniDFSNNTopology.NSConf nameservice : nnTopology.getNameservices()) {
|
|
|
+ String nsId = nameservice.getId();
|
|
|
+ nsIds.add(nameservice.getId());
|
|
|
+
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ !federation || nsId != null,
|
|
|
+ "if there is more than one NS, they must have names");
|
|
|
+
|
|
|
+ // First set up the configuration which all of the NNs
|
|
|
+ // need to have - have to do this a priori before starting
|
|
|
+ // *any* of the NNs, so they know to come up in standby.
|
|
|
+ List<String> nnIds = Lists.newArrayList();
|
|
|
+ // Iterate over the NNs in this nameservice
|
|
|
+ for (NNConf nn : nameservice.getNNs()) {
|
|
|
+ nnIds.add(nn.getNnId());
|
|
|
+
|
|
|
+ initNameNodeAddress(conf, nameservice.getId(), nn);
|
|
|
+ }
|
|
|
|
|
|
- /* For federated namenode initialize the address:port */
|
|
|
- private static void initFederatedNamenodeAddress(Configuration conf,
|
|
|
- String nameserviceId, int nnPort) {
|
|
|
- // Set nameserviceId specific key
|
|
|
- String key = DFSUtil.addKeySuffixes(
|
|
|
- DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId);
|
|
|
- conf.set(key, "127.0.0.1:0");
|
|
|
+ // If HA is enabled on this nameservice, enumerate all the namenodes
|
|
|
+ // in the configuration. Also need to set a shared edits dir
|
|
|
+ if (nnIds.size() > 1) {
|
|
|
+ conf.set(DFSUtil.addKeySuffixes(DFS_HA_NAMENODES_KEY, nameservice.getId()),
|
|
|
+ Joiner.on(",").join(nnIds));
|
|
|
+ if (manageNameDfsDirs) {
|
|
|
+ URI sharedEditsUri = fileAsURI(new File(base_dir, "shared-edits-" +
|
|
|
+ nnCounter + "-through-" + (nnCounter+nnIds.size()-1)));
|
|
|
+ // TODO in HDFS-1971: conf.set(DFS_NAMENODE_SHARED_EDITS_DIR_KEY, sharedEditsUri.toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- key = DFSUtil.addKeySuffixes(
|
|
|
- DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId);
|
|
|
- conf.set(key, "127.0.0.1:" + nnPort);
|
|
|
- }
|
|
|
-
|
|
|
- private void createFederationNamenodes(Configuration conf,
|
|
|
- Collection<String> nameserviceIds, boolean manageNameDfsDirs,
|
|
|
- boolean format, StartupOption operation, String clusterId)
|
|
|
- throws IOException {
|
|
|
- // Create namenodes in the cluster
|
|
|
- int nnCounter = 0;
|
|
|
- for (String nameserviceId : nameserviceIds) {
|
|
|
- createFederatedNameNode(nnCounter++, conf, numDataNodes, manageNameDfsDirs,
|
|
|
- format, operation, clusterId, nameserviceId);
|
|
|
+ // Now start all the NNs in this nameservice.
|
|
|
+ int i = 0;
|
|
|
+ for (NNConf nn : nameservice.getNNs()) {
|
|
|
+ initNameNodeConf(conf, nsId, nn.getNnId(), manageNameDfsDirs, nnCounter);
|
|
|
+
|
|
|
+ boolean formatThisOne = format;
|
|
|
+ if (format && i++ > 0) {
|
|
|
+ // Don't format the second NN in an HA setup - that
|
|
|
+ // would result in it having a different clusterID,
|
|
|
+ // block pool ID, etc. Instead, copy the name dirs
|
|
|
+ // from the first one.
|
|
|
+ formatThisOne = false;
|
|
|
+ copyNameDirs(getConfiguration(nnCounter - 1), conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ createNameNode(nnCounter++, conf, numDataNodes, formatThisOne,
|
|
|
+ operation, clusterId, nsId, nn.getNnId());
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+ if (federation) {
|
|
|
+ // If we have more than one nameservice, need to enumerate them in the
|
|
|
+ // config.
|
|
|
+ conf.set(DFS_FEDERATION_NAMESERVICES, Joiner.on(",").join(nsIds));
|
|
|
}
|
|
|
+
|
|
|
}
|
|
|
|
|
|
- private NameNode createNameNode(int nnIndex, Configuration conf,
|
|
|
- int numDataNodes, boolean manageNameDfsDirs, boolean format,
|
|
|
- StartupOption operation, String clusterId)
|
|
|
+ private void initNameNodeConf(Configuration conf,
|
|
|
+ String nameserviceId, String nnId,
|
|
|
+ boolean manageNameDfsDirs, int nnIndex)
|
|
|
throws IOException {
|
|
|
+ if (nameserviceId != null) {
|
|
|
+ conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
|
|
|
+ }
|
|
|
+ if (nnId != null) {
|
|
|
+ conf.set(DFS_HA_NAMENODE_ID_KEY, nnId);
|
|
|
+ }
|
|
|
+
|
|
|
if (manageNameDfsDirs) {
|
|
|
conf.set(DFS_NAMENODE_NAME_DIR_KEY,
|
|
|
fileAsURI(new File(base_dir, "name" + (2*nnIndex + 1)))+","+
|
|
@@ -628,7 +657,49 @@ public class MiniDFSCluster {
|
|
|
fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 1)))+","+
|
|
|
fileAsURI(new File(base_dir, "namesecondary" + (2*nnIndex + 2))));
|
|
|
}
|
|
|
-
|
|
|
+ }
|
|
|
+
|
|
|
+ private void copyNameDirs(Configuration srcConf, Configuration dstConf)
|
|
|
+ throws IOException {
|
|
|
+ Collection<URI> srcDirs = FSNamesystem.getNamespaceDirs(srcConf);
|
|
|
+ Collection<URI> dstDirs = FSNamesystem.getNamespaceDirs(dstConf);
|
|
|
+ URI srcDir = Lists.newArrayList(srcDirs).get(0);
|
|
|
+ FileSystem dstFS = FileSystem.getLocal(dstConf).getRaw();
|
|
|
+ for (URI dstDir : dstDirs) {
|
|
|
+ Preconditions.checkArgument(!dstDir.equals(srcDir));
|
|
|
+ Files.deleteRecursively(new File(dstDir));
|
|
|
+ LOG.info("Copying namedir from primary node dir "
|
|
|
+ + srcDir + " to " + dstDir);
|
|
|
+ FileUtil.copy(
|
|
|
+ new File(srcDir),
|
|
|
+ dstFS, new Path(dstDir), false, dstConf);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Initialize the address and port for this NameNode. In the
|
|
|
+ * non-federated case, the nameservice and namenode ID may be
|
|
|
+ * null.
|
|
|
+ */
|
|
|
+ private static void initNameNodeAddress(Configuration conf,
|
|
|
+ String nameserviceId, NNConf nnConf) {
|
|
|
+ // Set NN-specific specific key
|
|
|
+ String key = DFSUtil.addKeySuffixes(
|
|
|
+ DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId,
|
|
|
+ nnConf.getNnId());
|
|
|
+ conf.set(key, "127.0.0.1:" + nnConf.getHttpPort());
|
|
|
+
|
|
|
+ key = DFSUtil.addKeySuffixes(
|
|
|
+ DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId,
|
|
|
+ nnConf.getNnId());
|
|
|
+ conf.set(key, "127.0.0.1:" + nnConf.getIpcPort());
|
|
|
+ }
|
|
|
+
|
|
|
+ private void createNameNode(int nnIndex, Configuration conf,
|
|
|
+ int numDataNodes, boolean format, StartupOption operation,
|
|
|
+ String clusterId, String nameserviceId,
|
|
|
+ String nnId)
|
|
|
+ throws IOException {
|
|
|
// Format and clean out DataNode directories
|
|
|
if (format) {
|
|
|
DFSTestUtil.formatNameNode(conf);
|
|
@@ -642,23 +713,17 @@ public class MiniDFSCluster {
|
|
|
operation == StartupOption.FORMAT ||
|
|
|
operation == StartupOption.REGULAR) ?
|
|
|
new String[] {} : new String[] {operation.getName()};
|
|
|
- return NameNode.createNameNode(args, conf);
|
|
|
- }
|
|
|
-
|
|
|
- private void createFederatedNameNode(int nnIndex, Configuration conf,
|
|
|
- int numDataNodes, boolean manageNameDfsDirs, boolean format,
|
|
|
- StartupOption operation, String clusterId, String nameserviceId)
|
|
|
- throws IOException {
|
|
|
- conf.set(DFS_FEDERATION_NAMESERVICE_ID, nameserviceId);
|
|
|
- NameNode nn = createNameNode(nnIndex, conf, numDataNodes, manageNameDfsDirs,
|
|
|
- format, operation, clusterId);
|
|
|
+ NameNode nn = NameNode.createNameNode(args, conf);
|
|
|
+
|
|
|
+ // After the NN has started, set back the bound ports into
|
|
|
+ // the conf
|
|
|
conf.set(DFSUtil.addKeySuffixes(
|
|
|
- DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId), NameNode
|
|
|
+ DFS_NAMENODE_RPC_ADDRESS_KEY, nameserviceId, nnId), NameNode
|
|
|
.getHostPortString(nn.getNameNodeAddress()));
|
|
|
conf.set(DFSUtil.addKeySuffixes(
|
|
|
- DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId), NameNode
|
|
|
+ DFS_NAMENODE_HTTP_ADDRESS_KEY, nameserviceId, nnId), NameNode
|
|
|
.getHostPortString(nn.getHttpAddress()));
|
|
|
- DFSUtil.setGenericConf(conf, nameserviceId,
|
|
|
+ DFSUtil.setGenericConf(conf, nameserviceId, nnId,
|
|
|
DFS_NAMENODE_HTTP_ADDRESS_KEY);
|
|
|
nameNodes[nnIndex] = new NameNodeInfo(nn, new Configuration(conf));
|
|
|
}
|
|
@@ -1110,6 +1175,7 @@ public class MiniDFSCluster {
|
|
|
LOG.info("Shutting down the Mini HDFS Cluster");
|
|
|
shutdownDataNodes();
|
|
|
for (NameNodeInfo nnInfo : nameNodes) {
|
|
|
+ if (nnInfo == null) continue;
|
|
|
NameNode nameNode = nnInfo.nameNode;
|
|
|
if (nameNode != null) {
|
|
|
nameNode.stop();
|
|
@@ -1380,14 +1446,7 @@ public class MiniDFSCluster {
|
|
|
return false;
|
|
|
}
|
|
|
long[] sizes;
|
|
|
- try {
|
|
|
- sizes = nameNode.getRpcServer().getStats();
|
|
|
- } catch (IOException ioe) {
|
|
|
- // This method above should never throw.
|
|
|
- // It only throws IOE since it is exposed via RPC
|
|
|
- throw (AssertionError)(new AssertionError("Unexpected IOE thrown: "
|
|
|
- + StringUtils.stringifyException(ioe)).initCause(ioe));
|
|
|
- }
|
|
|
+ sizes = NameNodeAdapter.getStats(nameNode.getNamesystem());
|
|
|
boolean isUp = false;
|
|
|
synchronized (this) {
|
|
|
isUp = ((!nameNode.isInSafeMode() || !waitSafeMode) && sizes[0] != 0);
|
|
@@ -1497,6 +1556,22 @@ public class MiniDFSCluster {
|
|
|
public Collection<URI> getNameEditsDirs(int nnIndex) {
|
|
|
return FSNamesystem.getNamespaceEditsDirs(nameNodes[nnIndex].conf);
|
|
|
}
|
|
|
+
|
|
|
+ private HAServiceProtocol getHaServiceClient(int nnIndex) throws IOException {
|
|
|
+ InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
|
|
|
+ return RPC.getProxy(HAServiceProtocol.class,
|
|
|
+ HAServiceProtocol.versionID, addr, conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void transitionToActive(int nnIndex) throws IOException,
|
|
|
+ ServiceFailedException {
|
|
|
+ getHaServiceClient(nnIndex).transitionToActive();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void transitionToStandby(int nnIndex) throws IOException,
|
|
|
+ ServiceFailedException {
|
|
|
+ getHaServiceClient(nnIndex).transitionToActive();
|
|
|
+ }
|
|
|
|
|
|
/** Wait until the given namenode gets registration from all the datanodes */
|
|
|
public void waitActive(int nnIndex) throws IOException {
|
|
@@ -1504,6 +1579,7 @@ public class MiniDFSCluster {
|
|
|
return;
|
|
|
}
|
|
|
InetSocketAddress addr = nameNodes[nnIndex].nameNode.getServiceRpcAddress();
|
|
|
+ assert addr.getPort() != 0;
|
|
|
DFSClient client = new DFSClient(addr, conf);
|
|
|
|
|
|
// ensure all datanodes have registered and sent heartbeat to the namenode
|
|
@@ -1902,7 +1978,7 @@ public class MiniDFSCluster {
|
|
|
throws IOException {
|
|
|
if(!federation)
|
|
|
throw new IOException("cannot add namenode to non-federated cluster");
|
|
|
-
|
|
|
+
|
|
|
int nnIndex = nameNodes.length;
|
|
|
int numNameNodes = nameNodes.length + 1;
|
|
|
NameNodeInfo[] newlist = new NameNodeInfo[numNameNodes];
|
|
@@ -1913,10 +1989,13 @@ public class MiniDFSCluster {
|
|
|
String nameserviceIds = conf.get(DFS_FEDERATION_NAMESERVICES);
|
|
|
nameserviceIds += "," + nameserviceId;
|
|
|
conf.set(DFS_FEDERATION_NAMESERVICES, nameserviceIds);
|
|
|
-
|
|
|
- initFederatedNamenodeAddress(conf, nameserviceId, namenodePort);
|
|
|
- createFederatedNameNode(nnIndex, conf, numDataNodes, true, true, null,
|
|
|
- null, nameserviceId);
|
|
|
+
|
|
|
+ String nnId = null;
|
|
|
+ initNameNodeAddress(conf, nameserviceId,
|
|
|
+ new NNConf(nnId).setIpcPort(namenodePort));
|
|
|
+ initNameNodeConf(conf, nameserviceId, nnId, true, nnIndex);
|
|
|
+ createNameNode(nnIndex, conf, numDataNodes, true, null, null,
|
|
|
+ nameserviceId, nnId);
|
|
|
|
|
|
// Refresh datanodes with the newly started namenode
|
|
|
for (DataNodeProperties dn : dataNodes) {
|