|
@@ -24,9 +24,7 @@ import static org.junit.Assert.assertNotNull;
|
|
|
import static org.junit.Assert.assertNull;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.fail;
|
|
|
-import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Matchers.anyBoolean;
|
|
|
-import static org.mockito.Matchers.anyLong;
|
|
|
import static org.mockito.Matchers.anyString;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
import static org.mockito.Mockito.when;
|
|
@@ -81,6 +79,8 @@ import org.apache.hadoop.fs.StorageType;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
|
import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeAdminProperties;
|
|
|
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
|
@@ -89,11 +89,14 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.CombinedHostFileManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.HostConfigManager;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
|
|
|
import org.apache.hadoop.hdfs.tools.DFSck;
|
|
|
+import org.apache.hadoop.hdfs.util.HostsFileWriter;
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
@@ -1852,4 +1855,71 @@ public class TestFsck {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test(timeout = 60000)
|
|
|
+ public void testFsckUpgradeDomain() throws Exception {
|
|
|
+ testUpgradeDomain(false, false);
|
|
|
+ testUpgradeDomain(false, true);
|
|
|
+ testUpgradeDomain(true, false);
|
|
|
+ testUpgradeDomain(true, true);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void testUpgradeDomain(boolean defineUpgradeDomain,
|
|
|
+ boolean displayUpgradeDomain) throws Exception {
|
|
|
+ final short replFactor = 1;
|
|
|
+ final short numDN = 1;
|
|
|
+ final long blockSize = 512;
|
|
|
+ final long fileSize = 1024;
|
|
|
+ final String upgradeDomain = "ud1";
|
|
|
+ final String[] racks = {"/rack1"};
|
|
|
+ final String[] hosts = {"127.0.0.1"};
|
|
|
+ HostsFileWriter hostsFileWriter = new HostsFileWriter();
|
|
|
+ Configuration conf = new Configuration();
|
|
|
+ conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, replFactor);
|
|
|
+ if (defineUpgradeDomain) {
|
|
|
+ conf.setClass(DFSConfigKeys.DFS_NAMENODE_HOSTS_PROVIDER_CLASSNAME_KEY,
|
|
|
+ CombinedHostFileManager.class, HostConfigManager.class);
|
|
|
+ hostsFileWriter.initialize(conf, "temp/fsckupgradedomain");
|
|
|
+ }
|
|
|
+
|
|
|
+ MiniDFSCluster cluster;
|
|
|
+ DistributedFileSystem dfs;
|
|
|
+ cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDN).
|
|
|
+ hosts(hosts).racks(racks).build();
|
|
|
+ cluster.waitClusterUp();
|
|
|
+ dfs = cluster.getFileSystem();
|
|
|
+
|
|
|
+ // Configure the upgrade domain on the datanode
|
|
|
+ if (defineUpgradeDomain) {
|
|
|
+ DatanodeAdminProperties dnProp = new DatanodeAdminProperties();
|
|
|
+ DatanodeID datanodeID = cluster.getDataNodes().get(0).getDatanodeId();
|
|
|
+ dnProp.setHostName(datanodeID.getHostName());
|
|
|
+ dnProp.setPort(datanodeID.getXferPort());
|
|
|
+ dnProp.setUpgradeDomain(upgradeDomain);
|
|
|
+ hostsFileWriter.initIncludeHosts(new DatanodeAdminProperties[]{dnProp});
|
|
|
+ cluster.getFileSystem().refreshNodes();
|
|
|
+ }
|
|
|
+
|
|
|
+ // create files
|
|
|
+ final String testFile = new String("/testfile");
|
|
|
+ final Path path = new Path(testFile);
|
|
|
+ DFSTestUtil.createFile(dfs, path, fileSize, replFactor, 1000L);
|
|
|
+ DFSTestUtil.waitReplication(dfs, path, replFactor);
|
|
|
+ try {
|
|
|
+ String fsckOut = runFsck(conf, 0, true, testFile, "-files", "-blocks",
|
|
|
+ displayUpgradeDomain ? "-upgradedomains" : "-locations");
|
|
|
+ assertTrue(fsckOut.contains(NamenodeFsck.HEALTHY_STATUS));
|
|
|
+ String udValue = defineUpgradeDomain ? upgradeDomain :
|
|
|
+ NamenodeFsck.UNDEFINED;
|
|
|
+ assertEquals(displayUpgradeDomain,
|
|
|
+ fsckOut.contains("(ud=" + udValue + ")"));
|
|
|
+ } finally {
|
|
|
+ if (defineUpgradeDomain) {
|
|
|
+ hostsFileWriter.cleanup();
|
|
|
+ }
|
|
|
+ if (cluster != null) {
|
|
|
+ cluster.shutdown();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|