Browse Source

HDFS-6678. Merging change r1610549 from trunk to branch-2.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1610551 13f79535-47bb-0310-9956-ffa450edef68
Chris Nauroth 10 years ago
parent
commit
f7cb4c6e03

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -53,6 +53,9 @@ Release 2.6.0 - UNRELEASED
     HADOOP-8158. Interrupting hadoop fs -put from the command line
     HADOOP-8158. Interrupting hadoop fs -put from the command line
     causes a LeaseExpiredException. (daryn via harsh)
     causes a LeaseExpiredException. (daryn via harsh)
 
 
+    HDFS-6678. MiniDFSCluster may still be partially running after initialization
+    fails. (cnauroth)
+
 Release 2.5.0 - UNRELEASED
 Release 2.5.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 68 - 60
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -662,76 +662,84 @@ public class MiniDFSCluster {
       boolean checkDataNodeHostConfig,
       boolean checkDataNodeHostConfig,
       Configuration[] dnConfOverlays)
       Configuration[] dnConfOverlays)
   throws IOException {
   throws IOException {
-    ExitUtil.disableSystemExit();
+    boolean success = false;
+    try {
+      ExitUtil.disableSystemExit();
 
 
-    // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
-    FileSystem.enableSymlinks();
+      // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
+      FileSystem.enableSymlinks();
 
 
-    synchronized (MiniDFSCluster.class) {
-      instanceId = instanceCount++;
-    }
+      synchronized (MiniDFSCluster.class) {
+        instanceId = instanceCount++;
+      }
 
 
-    this.conf = conf;
-    base_dir = new File(determineDfsBaseDir());
-    data_dir = new File(base_dir, "data");
-    this.waitSafeMode = waitSafeMode;
-    this.checkExitOnShutdown = checkExitOnShutdown;
+      this.conf = conf;
+      base_dir = new File(determineDfsBaseDir());
+      data_dir = new File(base_dir, "data");
+      this.waitSafeMode = waitSafeMode;
+      this.checkExitOnShutdown = checkExitOnShutdown;
     
     
-    int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
-    conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
-    int safemodeExtension = conf.getInt(
-        DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY, 0);
-    conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, safemodeExtension);
-    conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
-    conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
-                   StaticMapping.class, DNSToSwitchMapping.class);
+      int replication = conf.getInt(DFS_REPLICATION_KEY, 3);
+      conf.setInt(DFS_REPLICATION_KEY, Math.min(replication, numDataNodes));
+      int safemodeExtension = conf.getInt(
+          DFS_NAMENODE_SAFEMODE_EXTENSION_TESTING_KEY, 0);
+      conf.setInt(DFS_NAMENODE_SAFEMODE_EXTENSION_KEY, safemodeExtension);
+      conf.setInt(DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY, 3); // 3 second
+      conf.setClass(NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, 
+                     StaticMapping.class, DNSToSwitchMapping.class);
     
     
-    // In an HA cluster, in order for the StandbyNode to perform checkpoints,
-    // it needs to know the HTTP port of the Active. So, if ephemeral ports
-    // are chosen, disable checkpoints for the test.
-    if (!nnTopology.allHttpPortsSpecified() &&
-        nnTopology.isHA()) {
-      LOG.info("MiniDFSCluster disabling checkpointing in the Standby node " +
-          "since no HTTP ports have been specified.");
-      conf.setBoolean(DFS_HA_STANDBY_CHECKPOINTS_KEY, false);
-    }
-    if (!nnTopology.allIpcPortsSpecified() &&
-        nnTopology.isHA()) {
-      LOG.info("MiniDFSCluster disabling log-roll triggering in the "
-          + "Standby node since no IPC ports have been specified.");
-      conf.setInt(DFS_HA_LOGROLL_PERIOD_KEY, -1);
-    }
+      // In an HA cluster, in order for the StandbyNode to perform checkpoints,
+      // it needs to know the HTTP port of the Active. So, if ephemeral ports
+      // are chosen, disable checkpoints for the test.
+      if (!nnTopology.allHttpPortsSpecified() &&
+          nnTopology.isHA()) {
+        LOG.info("MiniDFSCluster disabling checkpointing in the Standby node " +
+            "since no HTTP ports have been specified.");
+        conf.setBoolean(DFS_HA_STANDBY_CHECKPOINTS_KEY, false);
+      }
+      if (!nnTopology.allIpcPortsSpecified() &&
+          nnTopology.isHA()) {
+        LOG.info("MiniDFSCluster disabling log-roll triggering in the "
+            + "Standby node since no IPC ports have been specified.");
+        conf.setInt(DFS_HA_LOGROLL_PERIOD_KEY, -1);
+      }
     
     
-    federation = nnTopology.isFederated();
-    try {
-      createNameNodesAndSetConf(
-          nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
-          enableManagedDfsDirsRedundancy,
-          format, startOpt, clusterId, conf);
-    } catch (IOException ioe) {
-      LOG.error("IOE creating namenodes. Permissions dump:\n" +
-          createPermissionsDiagnosisString(data_dir));
-      throw ioe;
-    }
-    if (format) {
-      if (data_dir.exists() && !FileUtil.fullyDelete(data_dir)) {
-        throw new IOException("Cannot remove data directory: " + data_dir +
+      federation = nnTopology.isFederated();
+      try {
+        createNameNodesAndSetConf(
+            nnTopology, manageNameDfsDirs, manageNameDfsSharedDirs,
+            enableManagedDfsDirsRedundancy,
+            format, startOpt, clusterId, conf);
+      } catch (IOException ioe) {
+        LOG.error("IOE creating namenodes. Permissions dump:\n" +
             createPermissionsDiagnosisString(data_dir));
             createPermissionsDiagnosisString(data_dir));
+        throw ioe;
+      }
+      if (format) {
+        if (data_dir.exists() && !FileUtil.fullyDelete(data_dir)) {
+          throw new IOException("Cannot remove data directory: " + data_dir +
+              createPermissionsDiagnosisString(data_dir));
+        }
       }
       }
-    }
     
     
-    if (startOpt == StartupOption.RECOVER) {
-      return;
-    }
+      if (startOpt == StartupOption.RECOVER) {
+        return;
+      }
 
 
-    // Start the DataNodes
-    startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs,
-        dnStartOpt != null ? dnStartOpt : startOpt,
-        racks, hosts, simulatedCapacities, setupHostsFile,
-        checkDataNodeAddrConfig, checkDataNodeHostConfig, dnConfOverlays);
-    waitClusterUp();
-    //make sure ProxyUsers uses the latest conf
-    ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+      // Start the DataNodes
+      startDataNodes(conf, numDataNodes, storageType, manageDataDfsDirs,
+          dnStartOpt != null ? dnStartOpt : startOpt,
+          racks, hosts, simulatedCapacities, setupHostsFile,
+          checkDataNodeAddrConfig, checkDataNodeHostConfig, dnConfOverlays);
+      waitClusterUp();
+      //make sure ProxyUsers uses the latest conf
+      ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
+      success = true;
+    } finally {
+      if (!success) {
+        shutdown();
+      }
+    }
   }
   }
   
   
   /**
   /**