Browse Source

HDFS-11625. Ozone: Fix UT failures that caused by hard coded datanode data dirs. Contributed by Weiwei Yang.

Anu Engineer 8 years ago
parent
commit
bc2c5d2d61

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cblock/TestCBlockCLI.java

@@ -62,9 +62,11 @@ public class TestCBlockCLI {
 
   @AfterClass
   public static void clean() {
-    cBlockManager.stop();
-    cBlockManager.join();
-    cBlockManager.clean();
+    if (cBlockManager != null) {
+      cBlockManager.stop();
+      cBlockManager.join();
+      cBlockManager.clean();
+    }
   }
 
   @After

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockStatsMXBean.java

@@ -169,10 +169,9 @@ public class TestBlockStatsMXBean {
 
     storageTypeStats = storageTypeStatsMap.get(StorageType.ARCHIVE);
     assertEquals(3, storageTypeStats.getNodesInService());
-    String dataDir = cluster.getDataDirectory();
-    File dn1ArcVol1 = new File(dataDir, "data" + (3 * 0 + 2));
-    File dn2ArcVol1 = new File(dataDir, "data" + (3 * 1 + 2));
-    File dn3ArcVol1 = new File(dataDir, "data" + (3 * 2 + 2));
+    File dn1ArcVol1 = cluster.getInstanceStorageDir(0, 1);
+    File dn2ArcVol1 = cluster.getInstanceStorageDir(1, 1);
+    File dn3ArcVol1 = cluster.getInstanceStorageDir(2, 1);
     DataNodeTestUtils.injectDataDirFailure(dn1ArcVol1);
     DataNodeTestUtils.injectDataDirFailure(dn2ArcVol1);
     DataNodeTestUtils.injectDataDirFailure(dn3ArcVol1);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java

@@ -916,7 +916,7 @@ public class TestDataNodeHotSwapVolumes {
 
     DataNode dn = cluster.getDataNodes().get(0);
     final String oldDataDir = dn.getConf().get(DFS_DATANODE_DATA_DIR_KEY);
-    File dirToFail = new File(cluster.getDataDirectory(), "data1");
+    File dirToFail = cluster.getInstanceStorageDir(0, 0);
 
     FsVolumeImpl failedVolume = DataNodeTestUtils.getVolume(dn, dirToFail);
     assertTrue("No FsVolume was found for " + dirToFail,

+ 12 - 13
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -152,8 +152,7 @@ public class TestDataNodeVolumeFailure {
   @Test(timeout = 120000)
   public void testVolumeFailure() throws Exception {
     System.out.println("Data dir: is " +  dataDir.getPath());
-   
-    
+
     // Data dir structure is dataDir/data[1-4]/[current,tmp...]
     // data1,2 is for datanode 1, data2,3 - datanode2 
     String filename = "/test.txt";
@@ -168,7 +167,7 @@ public class TestDataNodeVolumeFailure {
    
     // fail the volume
     // delete/make non-writable one of the directories (failed volume)
-    data_fail = new File(dataDir, "data3");
+    data_fail = cluster.getInstanceStorageDir(1, 0);
     failedDir = MiniDFSCluster.getFinalizedDir(data_fail,
         cluster.getNamesystem().getBlockPoolId());
     if (failedDir.exists() &&
@@ -235,7 +234,7 @@ public class TestDataNodeVolumeFailure {
     DFSTestUtil.createFile(fs, file1, 1024, (short) 2, 1L);
     DFSTestUtil.waitReplication(fs, file1, (short) 2);
 
-    File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
+    File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
     DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
     DataNode dn0 = cluster.getDataNodes().get(0);
     DataNodeTestUtils.waitForDiskError(dn0,
@@ -294,8 +293,8 @@ public class TestDataNodeVolumeFailure {
   public void testDataNodeShutdownAfterNumFailedVolumeExceedsTolerated()
       throws Exception {
     // make both data directories to fail on dn0
-    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
-    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
+    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
     DataNodeTestUtils.injectDataDirFailure(dn0Vol1, dn0Vol2);
     DataNode dn0 = cluster.getDataNodes().get(0);
     DataNodeTestUtils.waitForDiskError(dn0,
@@ -314,8 +313,8 @@ public class TestDataNodeVolumeFailure {
   @Test
   public void testVolumeFailureRecoveredByHotSwappingVolume()
       throws Exception {
-    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
-    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
+    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
     final DataNode dn0 = cluster.getDataNodes().get(0);
     final String oldDataDirs = dn0.getConf().get(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -354,9 +353,9 @@ public class TestDataNodeVolumeFailure {
   @Test
   public void testTolerateVolumeFailuresAfterAddingMoreVolumes()
       throws Exception {
-    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
-    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
-    final File dn0VolNew = new File(dataDir, "data_new");
+    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
+    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
+    final File dn0VolNew = new File(cluster.getDataDirectory(), "data_new");
     final DataNode dn0 = cluster.getDataNodes().get(0);
     final String oldDataDirs = dn0.getConf().get(
         DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -401,8 +400,8 @@ public class TestDataNodeVolumeFailure {
     DFSTestUtil.waitReplication(fs, file1, (short)3);
 
     // Fail the first volume on both datanodes
-    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
-    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
+    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
     DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     Path file2 = new Path("/test2");

+ 16 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java

@@ -78,7 +78,6 @@ public class TestDataNodeVolumeFailureReporting {
   private FileSystem fs;
   private MiniDFSCluster cluster;
   private Configuration conf;
-  private String dataDir;
   private long volumeCapacity;
 
   // Sleep at least 3 seconds (a 1s heartbeat plus padding) to allow
@@ -134,10 +133,10 @@ public class TestDataNodeVolumeFailureReporting {
     final long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
     long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
 
-    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
-    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    File dn3Vol1 = new File(dataDir, "data"+(2*2+1));
-    File dn3Vol2 = new File(dataDir, "data"+(2*2+2));
+    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
+    File dn3Vol1 = cluster.getInstanceStorageDir(2, 0);
+    File dn3Vol2 = cluster.getInstanceStorageDir(2, 1);
 
     /*
      * Make the 1st volume directories on the first two datanodes
@@ -275,8 +274,8 @@ public class TestDataNodeVolumeFailureReporting {
 
     // Fail the first volume on both datanodes (we have to keep the 
     // third healthy so one node in the pipeline will not fail). 
-    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
-    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
+    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
     DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     Path file1 = new Path("/test1");
@@ -317,10 +316,10 @@ public class TestDataNodeVolumeFailureReporting {
     long origCapacity = DFSTestUtil.getLiveDatanodeCapacity(dm);
     long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
 
-    File dn1Vol1 = new File(dataDir, "data"+(4*0+1));
-    File dn1Vol2 = new File(dataDir, "data"+(4*0+2));
-    File dn2Vol1 = new File(dataDir, "data"+(4*1+1));
-    File dn2Vol2 = new File(dataDir, "data"+(4*1+2));
+    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn1Vol2 = cluster.getInstanceStorageDir(0, 1);
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
+    File dn2Vol2 = cluster.getInstanceStorageDir(1, 1);
 
     // Make the first two volume directories on the first two datanodes
     // non-accessible.
@@ -376,10 +375,10 @@ public class TestDataNodeVolumeFailureReporting {
 
     // Fail the first volume on both datanodes (we have to keep the
     // third healthy so one node in the pipeline will not fail).
-    File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
-    File dn1Vol2 = new File(dataDir, "data"+(2*0+2));
-    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    File dn2Vol2 = new File(dataDir, "data"+(2*1+2));
+    File dn1Vol1 = cluster.getInstanceStorageDir(0, 0);
+    File dn1Vol2 = cluster.getInstanceStorageDir(0, 1);
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
+    File dn2Vol2 = cluster.getInstanceStorageDir(1, 1);
     DataNodeTestUtils.injectDataDirFailure(dn1Vol1);
     DataNodeTestUtils.injectDataDirFailure(dn2Vol1);
 
@@ -528,8 +527,8 @@ public class TestDataNodeVolumeFailureReporting {
   @Test
   public void testHotSwapOutFailedVolumeAndReporting()
           throws Exception {
-    final File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
-    final File dn0Vol2 = new File(dataDir, "data" + (2 * 0 + 2));
+    final File dn0Vol1 = cluster.getInstanceStorageDir(0, 0);
+    final File dn0Vol2 = cluster.getInstanceStorageDir(0, 1);
     final DataNode dn0 = cluster.getDataNodes().get(0);
     final String oldDataDirs = dn0.getConf().get(
             DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
@@ -777,7 +776,6 @@ public class TestDataNodeVolumeFailureReporting {
         .storagesPerDatanode(storagesPerDatanode).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
-    dataDir = cluster.getDataDirectory();
     long dnCapacity = DFSTestUtil.getDatanodeCapacity(
         cluster.getNamesystem().getBlockManager().getDatanodeManager(), 0);
     volumeCapacity = dnCapacity / cluster.getStoragesPerDatanode();

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java

@@ -50,7 +50,6 @@ public class TestDataNodeVolumeFailureToleration {
   private FileSystem fs;
   private MiniDFSCluster cluster;
   private Configuration conf;
-  private String dataDir;
 
   // Sleep at least 3 seconds (a 1s heartbeat plus padding) to allow
   // for heartbeats to propagate from the datanodes to the namenode.
@@ -80,7 +79,6 @@ public class TestDataNodeVolumeFailureToleration {
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
     cluster.waitActive();
     fs = cluster.getFileSystem();
-    dataDir = cluster.getDataDirectory();
   }
 
   @After
@@ -161,7 +159,7 @@ public class TestDataNodeVolumeFailureToleration {
     long dnCapacity = DFSTestUtil.getDatanodeCapacity(dm, 0);
 
     // Fail a volume on the 2nd DN
-    File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
+    File dn2Vol1 = cluster.getInstanceStorageDir(1, 0);
     DataNodeTestUtils.injectDataDirFailure(dn2Vol1);
 
     // Should only get two replicas (the first DN and the 3rd)

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeMetrics.java

@@ -103,8 +103,7 @@ public class TestDataNodeVolumeMetrics {
       ArrayList<DataNode> dns = cluster.getDataNodes();
       assertTrue("DN1 should be up", dns.get(0).isDatanodeUp());
 
-      final String dataDir = cluster.getDataDirectory();
-      final File dn1Vol2 = new File(dataDir, "data2");
+      final File dn1Vol2 = cluster.getInstanceStorageDir(0, 0);
 
       DataNodeTestUtils.injectDataDirFailure(dn1Vol2);
       verifyDataNodeVolumeMetrics(fs, cluster, fileName);

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/diskbalancer/command/TestDiskBalancerCommand.java

@@ -438,13 +438,15 @@ public class TestDiskBalancerCommand {
     assertThat(
         outputs.get(3),
         is(allOf(containsString("DISK"),
-            containsString("/dfs/data/data1"),
+            containsString(cluster.getInstanceStorageDir(0, 0).getName()),
             containsString("0.00"),
             containsString("1.00"))));
+
+
     assertThat(
         outputs.get(4),
         is(allOf(containsString("DISK"),
-            containsString("/dfs/data/data2"),
+            containsString(cluster.getInstanceStorageDir(0, 1).getName()),
             containsString("0.00"),
             containsString("1.00"))));
   }

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java

@@ -319,11 +319,14 @@ public class TestDFSAdmin {
           containsString("FAILED: Change property " +
               DFS_DATANODE_DATA_DIR_KEY));
     }
-    assertThat(outs.get(offset + 1),
-        is(allOf(containsString("From:"), containsString("data1"),
-            containsString("data2"))));
+    File dnDir0 = cluster.getInstanceStorageDir(0, 0);
+    File dnDir1 = cluster.getInstanceStorageDir(0, 1);
+    assertThat(outs.get(offset + 1), is(allOf(containsString("From:"),
+                containsString(dnDir0.getName()),
+                containsString(dnDir1.getName()))));
     assertThat(outs.get(offset + 2),
-        is(not(anyOf(containsString("data1"), containsString("data2")))));
+        is(not(anyOf(containsString(dnDir0.getName()),
+            containsString(dnDir1.getName())))));
     assertThat(outs.get(offset + 2),
         is(allOf(containsString("To"), containsString("data_new"))));
   }

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/ozone/container/common/TestDatanodeStateMachine.java

@@ -303,7 +303,6 @@ public class TestDatanodeStateMachine {
     for (String name : new String[] {
         "",          // Empty
         "x..y",      // Invalid schema
-        "scm:",      // Missing port
         "scm:xyz",   // Invalid port
         "scm:123456" // Port out of range
     }) {