Explorar el Código

HDFS-13592. TestNameNodePrunesMissingStorages#testNameNodePrunesUnreportedStorages does not shut down cluster properly. Contributed by Anbang Hu.

(cherry picked from commit 57b893de3d36d20f65ee81b5cc3cfef12594b75b)
Inigo Goiri hace 7 años
padre
commit
d923e05798

+ 53 - 44
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java

@@ -383,51 +383,60 @@ public class TestNameNodePrunesMissingStorages {
         .Builder(conf).numDataNodes(1)
         .Builder(conf).numDataNodes(1)
         .storagesPerDatanode(2)
         .storagesPerDatanode(2)
         .build();
         .build();
-    // Create two files to ensure each storage has a block
-    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
-        102400, 102400, 102400, (short)1,
-        0x1BAD5EE);
-    DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
-        102400, 102400, 102400, (short)1,
-        0x1BAD5EED);
-    // Get the datanode storages and data directories
-    DataNode dn = cluster.getDataNodes().get(0);
-    BlockManager bm = cluster.getNameNode().getNamesystem().getBlockManager();
-    DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
-        getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
-    DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
-        dnDescriptor.getStorageInfos();
-    Collection<String> oldDirs =  new ArrayList<String>(dn.getConf().
-        getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
-    // Keep the first data directory and remove the second.
-    String newDirs = oldDirs.iterator().next();
-    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
-    // Restart the datanode with the new conf
-    cluster.stopDataNode(0);
-    cluster.startDataNodes(conf, 1, false, null, null);
-    dn = cluster.getDataNodes().get(0);
-    cluster.waitActive();
-    // Assert that the dnDescriptor has both the storages after restart
-    assertArrayEquals(dnStoragesInfosBeforeRestart,
-        dnDescriptor.getStorageInfos());
-    // Assert that the removed storage is marked as FAILED
-    // when DN heartbeats to the NN
-    int numFailedStoragesWithBlocks = 0;
-    DatanodeStorageInfo failedStorageInfo = null;
-    for (DatanodeStorageInfo dnStorageInfo: dnDescriptor.getStorageInfos()) {
-      if (dnStorageInfo.areBlocksOnFailedStorage()) {
-        numFailedStoragesWithBlocks++;
-        failedStorageInfo = dnStorageInfo;
+    try {
+      cluster.waitActive();
+      // Create two files to ensure each storage has a block
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
+          102400, 102400, 102400, (short)1,
+          0x1BAD5EE);
+      DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
+          102400, 102400, 102400, (short)1,
+          0x1BAD5EED);
+      // Get the datanode storages and data directories
+      DataNode dn = cluster.getDataNodes().get(0);
+      BlockManager bm =
+          cluster.getNameNode().getNamesystem().getBlockManager();
+      DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
+          getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
+      DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
+          dnDescriptor.getStorageInfos();
+      Collection<String> oldDirs =  new ArrayList<String>(dn.getConf().
+          getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
+      // Keep the first data directory and remove the second.
+      String newDirs = oldDirs.iterator().next();
+      conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
+      // Restart the datanode with the new conf
+      cluster.stopDataNode(0);
+      cluster.startDataNodes(conf, 1, false, null, null);
+      dn = cluster.getDataNodes().get(0);
+      cluster.waitActive();
+      // Assert that the dnDescriptor has both the storages after restart
+      assertArrayEquals(dnStoragesInfosBeforeRestart,
+          dnDescriptor.getStorageInfos());
+      // Assert that the removed storage is marked as FAILED
+      // when DN heartbeats to the NN
+      int numFailedStoragesWithBlocks = 0;
+      DatanodeStorageInfo failedStorageInfo = null;
+      for (DatanodeStorageInfo dnStorageInfo: dnDescriptor.getStorageInfos()) {
+        if (dnStorageInfo.areBlocksOnFailedStorage()) {
+          numFailedStoragesWithBlocks++;
+          failedStorageInfo = dnStorageInfo;
+        }
+      }
+      assertEquals(1, numFailedStoragesWithBlocks);
+      // Heartbeat manager removes the blocks associated with this failed
+      // storage
+      bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
+      assertTrue(!failedStorageInfo.areBlocksOnFailedStorage());
+      // pruneStorageMap removes the unreported storage
+      cluster.triggerHeartbeats();
+      // Assert that the unreported storage is pruned
+      assertEquals(DataNode.getStorageLocations(dn.getConf()).size(),
+          dnDescriptor.getStorageInfos().length);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
       }
       }
     }
     }
-    assertEquals(1, numFailedStoragesWithBlocks);
-    // Heartbeat manager removes the blocks associated with this failed storage
-    bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
-    assertTrue(!failedStorageInfo.areBlocksOnFailedStorage());
-    // pruneStorageMap removes the unreported storage
-    cluster.triggerHeartbeats();
-    // Assert that the unreported storage is pruned
-    assertEquals(DataNode.getStorageLocations(dn.getConf()).size(),
-        dnDescriptor.getStorageInfos().length);
   }
   }
 }
 }