|
@@ -24,12 +24,18 @@ import java.io.IOException;
|
|
|
import java.io.OutputStreamWriter;
|
|
|
import java.io.Writer;
|
|
|
import java.net.InetSocketAddress;
|
|
|
+import java.net.URI;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.nio.channels.Channels;
|
|
|
import java.nio.channels.ReadableByteChannel;
|
|
|
-import java.nio.file.Files;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Collection;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.Iterator;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
|
|
@@ -44,14 +50,14 @@ import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.DFSClient;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
+import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
+import org.apache.hadoop.hdfs.MiniDFSNNTopology;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
-import org.apache.hadoop.hdfs.server.aliasmap.InMemoryAliasMap;
|
|
|
-import org.apache.hadoop.hdfs.server.aliasmap.InMemoryLevelDBAliasMapServer;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
|
|
@@ -60,8 +66,10 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.ProvidedStorageMap;
|
|
|
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
|
|
|
import org.apache.hadoop.hdfs.server.common.blockaliasmap.BlockAliasMap;
|
|
|
import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.InMemoryLevelDBAliasMapClient;
|
|
|
+import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.LevelDBFileRegionAliasMap;
|
|
|
import org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
|
|
@@ -71,6 +79,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.StorageReport;
|
|
|
+import org.apache.hadoop.ipc.RemoteException;
|
|
|
+import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
@@ -80,6 +90,12 @@ import org.junit.rules.TestName;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST;
|
|
|
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LEVELDB_PATH;
|
|
|
+import static org.apache.hadoop.hdfs.server.common.Util.fileAsURI;
|
|
|
import static org.apache.hadoop.hdfs.server.common.blockaliasmap.impl.TextFileRegionAliasMap.fileNameFromBlockPoolID;
|
|
|
import static org.apache.hadoop.net.NodeBase.PATH_SEPARATOR_STR;
|
|
|
import static org.junit.Assert.*;
|
|
@@ -106,6 +122,7 @@ public class ITestProvidedImplementation {
|
|
|
private final int baseFileLen = 1024;
|
|
|
private long providedDataSize = 0;
|
|
|
private final String bpid = "BP-1234-10.1.1.1-1224";
|
|
|
+ private static final String clusterID = "CID-PROVIDED";
|
|
|
|
|
|
private Configuration conf;
|
|
|
private MiniDFSCluster cluster;
|
|
@@ -214,36 +231,78 @@ public class ITestProvidedImplementation {
|
|
|
StorageType[] storageTypes,
|
|
|
StorageType[][] storageTypesPerDatanode,
|
|
|
boolean doFormat, String[] racks) throws IOException {
|
|
|
+ startCluster(nspath, numDatanodes,
|
|
|
+ storageTypes, storageTypesPerDatanode,
|
|
|
+ doFormat, racks, null,
|
|
|
+ new MiniDFSCluster.Builder(conf));
|
|
|
+ }
|
|
|
+
|
|
|
+ void startCluster(Path nspath, int numDatanodes,
|
|
|
+ StorageType[] storageTypes,
|
|
|
+ StorageType[][] storageTypesPerDatanode,
|
|
|
+ boolean doFormat, String[] racks,
|
|
|
+ MiniDFSNNTopology topo,
|
|
|
+ MiniDFSCluster.Builder builder) throws IOException {
|
|
|
conf.set(DFS_NAMENODE_NAME_DIR_KEY, nspath.toString());
|
|
|
|
|
|
+ builder.format(doFormat)
|
|
|
+ .manageNameDfsDirs(doFormat)
|
|
|
+ .numDataNodes(numDatanodes)
|
|
|
+ .racks(racks);
|
|
|
if (storageTypesPerDatanode != null) {
|
|
|
- cluster = new MiniDFSCluster.Builder(conf)
|
|
|
- .format(doFormat)
|
|
|
- .manageNameDfsDirs(doFormat)
|
|
|
- .numDataNodes(numDatanodes)
|
|
|
- .storageTypes(storageTypesPerDatanode)
|
|
|
- .racks(racks)
|
|
|
- .build();
|
|
|
+ builder.storageTypes(storageTypesPerDatanode);
|
|
|
} else if (storageTypes != null) {
|
|
|
- cluster = new MiniDFSCluster.Builder(conf)
|
|
|
- .format(doFormat)
|
|
|
- .manageNameDfsDirs(doFormat)
|
|
|
- .numDataNodes(numDatanodes)
|
|
|
- .storagesPerDatanode(storageTypes.length)
|
|
|
- .storageTypes(storageTypes)
|
|
|
- .racks(racks)
|
|
|
- .build();
|
|
|
- } else {
|
|
|
- cluster = new MiniDFSCluster.Builder(conf)
|
|
|
- .format(doFormat)
|
|
|
- .manageNameDfsDirs(doFormat)
|
|
|
- .numDataNodes(numDatanodes)
|
|
|
- .racks(racks)
|
|
|
- .build();
|
|
|
+ builder.storagesPerDatanode(storageTypes.length)
|
|
|
+ .storageTypes(storageTypes);
|
|
|
}
|
|
|
+ if (topo != null) {
|
|
|
+ builder.nnTopology(topo);
|
|
|
+ // If HA or Federation is enabled and formatting is set to false,
|
|
|
+ // copy the FSImage to all Namenode directories.
|
|
|
+ if ((topo.isHA() || topo.isFederated()) && !doFormat) {
|
|
|
+ builder.manageNameDfsDirs(true);
|
|
|
+ builder.enableManagedDfsDirsRedundancy(false);
|
|
|
+ builder.manageNameDfsSharedDirs(false);
|
|
|
+ List<File> nnDirs =
|
|
|
+ getProvidedNamenodeDirs(MiniDFSCluster.getBaseDirectory(), topo);
|
|
|
+ for (File nnDir : nnDirs) {
|
|
|
+ MiniDFSCluster.copyNameDirs(
|
|
|
+ Collections.singletonList(nspath.toUri()),
|
|
|
+ Collections.singletonList(fileAsURI(nnDir)),
|
|
|
+ conf);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ cluster = builder.build();
|
|
|
cluster.waitActive();
|
|
|
}
|
|
|
|
|
|
+ private static List<File> getProvidedNamenodeDirs(String baseDir,
|
|
|
+ MiniDFSNNTopology topo) {
|
|
|
+ List<File> nnDirs = new ArrayList<>();
|
|
|
+ int nsCounter = 0;
|
|
|
+ for (MiniDFSNNTopology.NSConf nsConf : topo.getNameservices()) {
|
|
|
+ int nnCounter = nsCounter;
|
|
|
+ for (MiniDFSNNTopology.NNConf nnConf : nsConf.getNNs()) {
|
|
|
+ if (providedNameservice.equals(nsConf.getId())) {
|
|
|
+ // only add the first one
|
|
|
+ File[] nnFiles =
|
|
|
+ MiniDFSCluster.getNameNodeDirectory(
|
|
|
+ baseDir, nsCounter, nnCounter);
|
|
|
+ if (nnFiles == null || nnFiles.length == 0) {
|
|
|
+ throw new RuntimeException("Failed to get a location for the"
|
|
|
+ + "Namenode directory for namespace: " + nsConf.getId()
|
|
|
+ + " and namenodeId: " + nnConf.getNnId());
|
|
|
+ }
|
|
|
+ nnDirs.add(nnFiles[0]);
|
|
|
+ }
|
|
|
+ nnCounter++;
|
|
|
+ }
|
|
|
+ nsCounter = nnCounter;
|
|
|
+ }
|
|
|
+ return nnDirs;
|
|
|
+ }
|
|
|
+
|
|
|
@Test(timeout=20000)
|
|
|
public void testLoadImage() throws Exception {
|
|
|
final long seed = r.nextLong();
|
|
@@ -405,8 +464,8 @@ public class ITestProvidedImplementation {
|
|
|
return ret;
|
|
|
}
|
|
|
|
|
|
- private void verifyFileSystemContents() throws Exception {
|
|
|
- FileSystem fs = cluster.getFileSystem();
|
|
|
+ private void verifyFileSystemContents(int nnIndex) throws Exception {
|
|
|
+ FileSystem fs = cluster.getFileSystem(nnIndex);
|
|
|
int count = 0;
|
|
|
// read NN metadata, verify contents match
|
|
|
for (TreePath e : new FSTreeWalk(providedPath, conf)) {
|
|
@@ -766,41 +825,255 @@ public class ITestProvidedImplementation {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- @Test
|
|
|
- public void testInMemoryAliasMap() throws Exception {
|
|
|
- conf.setClass(ImageWriter.Options.UGI_CLASS,
|
|
|
- FsUGIResolver.class, UGIResolver.class);
|
|
|
+ private File createInMemoryAliasMapImage() throws Exception {
|
|
|
+ conf.setClass(ImageWriter.Options.UGI_CLASS, FsUGIResolver.class,
|
|
|
+ UGIResolver.class);
|
|
|
conf.setClass(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_CLASS,
|
|
|
InMemoryLevelDBAliasMapClient.class, BlockAliasMap.class);
|
|
|
- conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
|
|
|
- "localhost:32445");
|
|
|
+ conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS, "localhost:32445");
|
|
|
File tempDirectory =
|
|
|
- Files.createTempDirectory("in-memory-alias-map").toFile();
|
|
|
- File leveDBPath = new File(tempDirectory, bpid);
|
|
|
- leveDBPath.mkdirs();
|
|
|
- conf.set(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
|
|
|
+ new File(new Path(nnDirPath, "in-memory-alias-map").toUri());
|
|
|
+ File levelDBDir = new File(tempDirectory, bpid);
|
|
|
+ levelDBDir.mkdirs();
|
|
|
+ conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
|
|
|
tempDirectory.getAbsolutePath());
|
|
|
- conf.setBoolean(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
|
|
|
conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
|
|
|
- InMemoryLevelDBAliasMapServer levelDBAliasMapServer =
|
|
|
- new InMemoryLevelDBAliasMapServer(InMemoryAliasMap::init, bpid);
|
|
|
- levelDBAliasMapServer.setConf(conf);
|
|
|
- levelDBAliasMapServer.start();
|
|
|
+ conf.set(DFS_PROVIDED_ALIASMAP_LEVELDB_PATH,
|
|
|
+ tempDirectory.getAbsolutePath());
|
|
|
|
|
|
createImage(new FSTreeWalk(providedPath, conf),
|
|
|
nnDirPath,
|
|
|
- FixedBlockResolver.class, "",
|
|
|
- InMemoryLevelDBAliasMapClient.class);
|
|
|
- levelDBAliasMapServer.close();
|
|
|
+ FixedBlockResolver.class, clusterID,
|
|
|
+ LevelDBFileRegionAliasMap.class);
|
|
|
+
|
|
|
+ return tempDirectory;
|
|
|
+ }
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testInMemoryAliasMap() throws Exception {
|
|
|
+ File aliasMapImage = createInMemoryAliasMapImage();
|
|
|
// start cluster with two datanodes,
|
|
|
// each with 1 PROVIDED volume and other DISK volume
|
|
|
+ conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
|
|
|
startCluster(nnDirPath, 2,
|
|
|
new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
|
|
|
null, false);
|
|
|
- verifyFileSystemContents();
|
|
|
- FileUtils.deleteDirectory(tempDirectory);
|
|
|
+ verifyFileSystemContents(0);
|
|
|
+ FileUtils.deleteDirectory(aliasMapImage);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Find a free port that hasn't been assigned yet.
|
|
|
+ *
|
|
|
+ * @param usedPorts set of ports that have already been assigned.
|
|
|
+ * @param maxTrials maximum number of random ports to try before failure.
|
|
|
+ * @return an unassigned port.
|
|
|
+ */
|
|
|
+ private int getUnAssignedPort(Set<Integer> usedPorts, int maxTrials) {
|
|
|
+ int count = 0;
|
|
|
+ while (count < maxTrials) {
|
|
|
+ int port = NetUtils.getFreeSocketPort();
|
|
|
+ if (usedPorts.contains(port)) {
|
|
|
+ count++;
|
|
|
+ } else {
|
|
|
+ return port;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return -1;
|
|
|
+ }
|
|
|
+
|
|
|
+ private static String providedNameservice;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Extends the {@link MiniDFSCluster.Builder} to create instances of
|
|
|
+ * {@link MiniDFSClusterBuilderAliasMap}.
|
|
|
+ */
|
|
|
+ private static class MiniDFSClusterBuilderAliasMap
|
|
|
+ extends MiniDFSCluster.Builder {
|
|
|
+
|
|
|
+ MiniDFSClusterBuilderAliasMap(Configuration conf) {
|
|
|
+ super(conf);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public MiniDFSCluster build() throws IOException {
|
|
|
+ return new MiniDFSClusterAliasMap(this);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Extends {@link MiniDFSCluster} to correctly configure the InMemoryAliasMap.
|
|
|
+ */
|
|
|
+ private static class MiniDFSClusterAliasMap extends MiniDFSCluster {
|
|
|
+
|
|
|
+ private Map<String, Collection<URI>> formattedDirsByNamespaceId;
|
|
|
+ private Set<Integer> completedNNs;
|
|
|
+
|
|
|
+ MiniDFSClusterAliasMap(MiniDFSCluster.Builder builder) throws IOException {
|
|
|
+ super(builder);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected void initNameNodeConf(Configuration conf, String nameserviceId,
|
|
|
+ int nsIndex, String nnId, boolean manageNameDfsDirs,
|
|
|
+ boolean enableManagedDfsDirsRedundancy, int nnIndex)
|
|
|
+ throws IOException {
|
|
|
+
|
|
|
+ if (formattedDirsByNamespaceId == null) {
|
|
|
+ formattedDirsByNamespaceId = new HashMap<>();
|
|
|
+ completedNNs = new HashSet<>();
|
|
|
+ }
|
|
|
+
|
|
|
+ super.initNameNodeConf(conf, nameserviceId, nsIndex, nnId,
|
|
|
+ manageNameDfsDirs, enableManagedDfsDirsRedundancy, nnIndex);
|
|
|
+
|
|
|
+ if (providedNameservice.equals(nameserviceId)) {
|
|
|
+ // configure the InMemoryAliasMp.
|
|
|
+ conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
|
|
|
+ String directory = conf.get(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
|
|
|
+ if (directory == null) {
|
|
|
+ throw new IllegalArgumentException("In-memory alias map configured"
|
|
|
+ + "with the proper location; Set "
|
|
|
+ + DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR);
|
|
|
+ }
|
|
|
+ // get the name of the directory (final component in path) used for map.
|
|
|
+ // Assume that the aliasmap configured with the same final component
|
|
|
+ // name in all Namenodes but is located in the path specified by
|
|
|
+ // DFS_NAMENODE_NAME_DIR_KEY
|
|
|
+ String dirName = new Path(directory).getName();
|
|
|
+ String nnDir =
|
|
|
+ conf.getTrimmedStringCollection(DFS_NAMENODE_NAME_DIR_KEY)
|
|
|
+ .iterator().next();
|
|
|
+ conf.set(DFS_PROVIDED_ALIASMAP_INMEMORY_LEVELDB_DIR,
|
|
|
+ new File(new Path(nnDir, dirName).toUri()).getAbsolutePath());
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, true);
|
|
|
+ } else {
|
|
|
+ if (!completedNNs.contains(nnIndex)) {
|
|
|
+ // format the NN directories for non-provided namespaces
|
|
|
+ // if the directory for a namespace has been formatted, copy it over.
|
|
|
+ Collection<URI> namespaceDirs = FSNamesystem.getNamespaceDirs(conf);
|
|
|
+ if (formattedDirsByNamespaceId.containsKey(nameserviceId)) {
|
|
|
+ copyNameDirs(formattedDirsByNamespaceId.get(nameserviceId),
|
|
|
+ namespaceDirs, conf);
|
|
|
+ } else {
|
|
|
+ for (URI nameDirUri : namespaceDirs) {
|
|
|
+ File nameDir = new File(nameDirUri);
|
|
|
+ if (nameDir.exists() && !FileUtil.fullyDelete(nameDir)) {
|
|
|
+ throw new IOException("Could not fully delete " + nameDir);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ HdfsServerConstants.StartupOption.FORMAT.setClusterId(clusterID);
|
|
|
+ DFSTestUtil.formatNameNode(conf);
|
|
|
+ formattedDirsByNamespaceId.put(nameserviceId, namespaceDirs);
|
|
|
+ }
|
|
|
+ conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_PROVIDED_ENABLED, false);
|
|
|
+ completedNNs.add(nnIndex);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Configures the addresseses of the InMemoryAliasMap.
|
|
|
+ *
|
|
|
+ * @param topology the MiniDFS topology to use.
|
|
|
+ * @param providedNameservice the nameservice id that supports provided.
|
|
|
+ */
|
|
|
+ private void configureAliasMapAddresses(MiniDFSNNTopology topology,
|
|
|
+ String providedNameservice) {
|
|
|
+ conf.unset(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS);
|
|
|
+ Set<Integer> assignedPorts = new HashSet<>();
|
|
|
+ for (MiniDFSNNTopology.NSConf nsConf : topology.getNameservices()) {
|
|
|
+ for (MiniDFSNNTopology.NNConf nnConf : nsConf.getNNs()) {
|
|
|
+ if (providedNameservice.equals(nsConf.getId())) {
|
|
|
+ String key =
|
|
|
+ DFSUtil.addKeySuffixes(DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS,
|
|
|
+ nsConf.getId(), nnConf.getNnId());
|
|
|
+ int port = getUnAssignedPort(assignedPorts, 10);
|
|
|
+ if (port == -1) {
|
|
|
+ throw new RuntimeException("No free ports available");
|
|
|
+ }
|
|
|
+ assignedPorts.add(port);
|
|
|
+ conf.set(key, "127.0.0.1:" + port);
|
|
|
+
|
|
|
+ String binHostKey =
|
|
|
+ DFSUtil.addKeySuffixes(
|
|
|
+ DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_BIND_HOST,
|
|
|
+ nsConf.getId(), nnConf.getNnId());
|
|
|
+ conf.set(binHostKey, "0.0.0.0");
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify the mounted contents of the Filesystem.
|
|
|
+ *
|
|
|
+ * @param topology the topology of the cluster.
|
|
|
+ * @param providedNameservice the namespace id of the provided namenodes.
|
|
|
+ * @throws Exception
|
|
|
+ */
|
|
|
+ private void verifyPathsWithHAFailoverIfNecessary(MiniDFSNNTopology topology,
|
|
|
+ String providedNameservice) throws Exception {
|
|
|
+ List<Integer> nnIndexes = cluster.getNNIndexes(providedNameservice);
|
|
|
+ if (topology.isHA()) {
|
|
|
+ int nn1 = nnIndexes.get(0);
|
|
|
+ int nn2 = nnIndexes.get(1);
|
|
|
+ try {
|
|
|
+ verifyFileSystemContents(nn1);
|
|
|
+ fail("Read operation should fail as no Namenode is active");
|
|
|
+ } catch (RemoteException e) {
|
|
|
+ LOG.info("verifyPaths failed!. Expected exception: {}" + e);
|
|
|
+ }
|
|
|
+ cluster.transitionToActive(nn1);
|
|
|
+ LOG.info("Verifying data from NN with index = {}", nn1);
|
|
|
+ verifyFileSystemContents(nn1);
|
|
|
+ // transition to the second namenode.
|
|
|
+ cluster.transitionToStandby(nn1);
|
|
|
+ cluster.transitionToActive(nn2);
|
|
|
+ LOG.info("Verifying data from NN with index = {}", nn2);
|
|
|
+ verifyFileSystemContents(nn2);
|
|
|
+
|
|
|
+ cluster.shutdownNameNodes();
|
|
|
+ try {
|
|
|
+ verifyFileSystemContents(nn2);
|
|
|
+ fail("Read operation should fail as no Namenode is active");
|
|
|
+ } catch (NullPointerException e) {
|
|
|
+ LOG.info("verifyPaths failed!. Expected exception: {}" + e);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ verifyFileSystemContents(nnIndexes.get(0));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testInMemoryAliasMapMultiTopologies() throws Exception {
|
|
|
+ MiniDFSNNTopology[] topologies =
|
|
|
+ new MiniDFSNNTopology[] {
|
|
|
+ MiniDFSNNTopology.simpleHATopology(),
|
|
|
+ MiniDFSNNTopology.simpleFederatedTopology(3),
|
|
|
+ MiniDFSNNTopology.simpleHAFederatedTopology(3)
|
|
|
+ };
|
|
|
+
|
|
|
+ for (MiniDFSNNTopology topology : topologies) {
|
|
|
+ LOG.info("Starting test with topology with HA = {}, federation = {}",
|
|
|
+ topology.isHA(), topology.isFederated());
|
|
|
+ setSeed();
|
|
|
+ createInMemoryAliasMapImage();
|
|
|
+ conf.setBoolean(DFS_PROVIDED_ALIASMAP_INMEMORY_ENABLED, true);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_PROVIDED_ALIASMAP_LOAD_RETRIES, 10);
|
|
|
+ providedNameservice = topology.getNameservices().get(0).getId();
|
|
|
+ // configure the AliasMap addresses
|
|
|
+ configureAliasMapAddresses(topology, providedNameservice);
|
|
|
+ startCluster(nnDirPath, 2,
|
|
|
+ new StorageType[] {StorageType.PROVIDED, StorageType.DISK},
|
|
|
+ null, false, null, topology,
|
|
|
+ new MiniDFSClusterBuilderAliasMap(conf));
|
|
|
+
|
|
|
+ verifyPathsWithHAFailoverIfNecessary(topology, providedNameservice);
|
|
|
+ shutdown();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private DatanodeDescriptor getDatanodeDescriptor(DatanodeManager dnm,
|
|
@@ -919,7 +1192,7 @@ public class ITestProvidedImplementation {
|
|
|
startCluster(nnDirPath, racks.length,
|
|
|
new StorageType[]{StorageType.PROVIDED, StorageType.DISK},
|
|
|
null, false, racks);
|
|
|
- verifyFileSystemContents();
|
|
|
+ verifyFileSystemContents(0);
|
|
|
setAndUnsetReplication("/" + filePrefix + (numFiles - 1) + fileSuffix);
|
|
|
cluster.shutdown();
|
|
|
}
|