Browse Source

HADOOP-8562. Merge r1457069 for HDFS-4604, r1459514 for YARN-488, r1459520 for YARN-490, r1459526 for YARN-491, r1459544 for MAPREDUCE-5078 and r1459573 for HDFS-4607

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1485911 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
02205fa797
11 changed files with 68 additions and 42 deletions
  1. 5 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 2 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java
  3. 20 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  4. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  5. 4 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  6. 6 0
      hadoop-yarn-project/CHANGES.txt
  7. 7 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  8. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
  9. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  10. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java
  11. 7 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

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

@@ -358,6 +358,11 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4603. TestMiniDFSCluster fails on Windows. (Ivan Mitic via suresh)
 
+    HDFS-4604. TestJournalNode fails on Windows. (Ivan Mitic via suresh)
+
+    HDFS-4607.  In TestGetConf.testGetSpecificKey(), use a platform-specific
+    line separator; otherwise, it fails on Windows.  (Ivan Mitic via szetszwo)
+
 Release 2.0.4-alpha - 2013-04-25
 
   INCOMPATIBLE CHANGES

+ 2 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNode.java

@@ -35,6 +35,7 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.metrics2.source.JvmMetrics;
 import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
@@ -82,7 +83,6 @@ public class JournalNode implements Tool, Configurable {
     return journal;
   }
 
-
   @Override
   public void setConf(Configuration conf) {
     this.conf = conf;
@@ -97,21 +97,9 @@ public class JournalNode implements Tool, Configurable {
           "Journal dir '" + dir + "' should be an absolute path");
     }
 
-    if (!dir.exists() && !dir.mkdirs()) {
-      throw new IOException("Could not create journal dir '" +
-          dir + "'");
-    } else if (!dir.isDirectory()) {
-      throw new IOException("Journal directory '" + dir + "' is not " +
-          "a directory");
-    }
-    
-    if (!dir.canWrite()) {
-      throw new IOException("Unable to write to journal dir '" +
-          dir + "'");
-    }
+    DiskChecker.checkDir(dir);
   }
 
-
   @Override
   public Configuration getConf() {
     return conf;

+ 20 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.MetricsAsserts;
+import org.apache.hadoop.util.Shell;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -65,6 +66,8 @@ public class TestJournalNode {
   private Configuration conf = new Configuration();
   private IPCLoggerChannel ch;
   private String journalId;
+  private File TEST_BUILD_DATA =
+      new File(System.getProperty("test.build.data", "build/test/data"));
 
   static {
     // Avoid an error when we double-initialize JvmMetrics
@@ -96,7 +99,7 @@ public class TestJournalNode {
     jn.stop(0);
   }
   
-  @Test
+  @Test(timeout=100000)
   public void testJournal() throws Exception {
     MetricsRecordBuilder metrics = MetricsAsserts.getMetrics(
         journal.getMetricsForTests().getName());
@@ -129,7 +132,7 @@ public class TestJournalNode {
   }
   
   
-  @Test
+  @Test(timeout=100000)
   public void testReturnsSegmentInfoAtEpochTransition() throws Exception {
     ch.newEpoch(1).get();
     ch.setEpoch(1);
@@ -157,7 +160,7 @@ public class TestJournalNode {
     assertEquals(1, response.getLastSegmentTxId());
   }
   
-  @Test
+  @Test(timeout=100000)
   public void testHttpServer() throws Exception {
     InetSocketAddress addr = jn.getBoundHttpAddress();
     assertTrue(addr.getPort() > 0);
@@ -210,7 +213,7 @@ public class TestJournalNode {
    * Test that the JournalNode performs correctly as a Paxos
    * <em>Acceptor</em> process.
    */
-  @Test
+  @Test(timeout=100000)
   public void testAcceptRecoveryBehavior() throws Exception {
     // We need to run newEpoch() first, or else we have no way to distinguish
     // different proposals for the same decision.
@@ -270,20 +273,27 @@ public class TestJournalNode {
     }
   }
   
-  @Test
+  @Test(timeout=100000)
   public void testFailToStartWithBadConfig() throws Exception {
     Configuration conf = new Configuration();
     conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "non-absolute-path");
     assertJNFailsToStart(conf, "should be an absolute path");
     
     // Existing file which is not a directory 
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/dev/null");
-    assertJNFailsToStart(conf, "is not a directory");
+    File existingFile = new File(TEST_BUILD_DATA, "testjournalnodefile");
+    assertTrue(existingFile.createNewFile());
+    try {
+      conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
+          existingFile.getAbsolutePath());
+      assertJNFailsToStart(conf, "Not a directory");
+    } finally {
+      existingFile.delete();
+    }
     
     // Directory which cannot be created
-    conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY, "/proc/does-not-exist");
-    assertJNFailsToStart(conf, "Could not create");
-
+    conf.set(DFSConfigKeys.DFS_JOURNALNODE_EDITS_DIR_KEY,
+        Shell.WINDOWS ? "\\\\cannotBeCreated" : "/proc/does-not-exist");
+    assertJNFailsToStart(conf, "Can not create directory");
   }
 
   private static void assertJNFailsToStart(Configuration conf,

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

@@ -210,6 +210,9 @@ Release 2.0.5-beta - UNRELEASED
 
     MAPREDUCE-4898. FileOutputFormat.checkOutputSpecs and 
     FileOutputFormat.setOutputPath incompatible with MR1. (rkanter via tucu)
+    
+    MAPREDUCE-5078. TestMRAppMaster fails on Windows due to mismatched path
+    separators. (Chris Nauroth via sseth)
 
     MAPREDUCE-4932. mapreduce.job#getTaskCompletionEvents incompatible with 
     Hadoop 1. (rkanter via tucu)

+ 4 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java

@@ -116,8 +116,10 @@ public class TestMRAppMaster {
     JobConf conf = new JobConf();
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, stagingDir);
     MRAppMaster.initAndStartAppMaster(appMaster, conf, userName);
-    assertEquals(stagingDir + Path.SEPARATOR + userName + Path.SEPARATOR
-        + ".staging", appMaster.stagingDirPath.toString());
+    Path userPath = new Path(stagingDir, userName);
+    Path userStagingPath = new Path(userPath, ".staging");
+    assertEquals(userStagingPath.toString(),
+      appMaster.stagingDirPath.toString());
   }
   
   @Test

+ 6 - 0
hadoop-yarn-project/CHANGES.txt

@@ -369,6 +369,12 @@ Release 2.0.5-beta - UNRELEASED
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
 
+    YARN-488. TestContainerManagerSecurity fails on Windows. (Chris Nauroth
+    via hitesh)
+
+    YARN-490. TestDistributedShell fails on Windows. (Chris Nauroth via hitesh)
+
+    YARN-491. TestContainerLogsPage fails on Windows. (Chris Nauroth via hitesh)
 
 Release 2.0.4-alpha - 2013-04-25 
 

+ 7 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.applications.distributedshell;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -39,6 +40,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
@@ -482,14 +484,15 @@ public class Client extends YarnClientImpl {
     // It should be provided out of the box. 
     // For now setting all required classpaths including
     // the classpath to "." for the application jar
-    StringBuilder classPathEnv = new StringBuilder("${CLASSPATH}:./*");
+    StringBuilder classPathEnv = new StringBuilder(Environment.CLASSPATH.$())
+      .append(File.pathSeparatorChar).append("./*");
     for (String c : conf.getStrings(
         YarnConfiguration.YARN_APPLICATION_CLASSPATH,
         YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH)) {
-      classPathEnv.append(':');
+      classPathEnv.append(File.pathSeparatorChar);
       classPathEnv.append(c.trim());
     }
-    classPathEnv.append(":./log4j.properties");
+    classPathEnv.append(File.pathSeparatorChar).append("./log4j.properties");
 
     // add the runtime classpath needed for tests to work
     if (conf.getBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, false)) {
@@ -506,7 +509,7 @@ public class Client extends YarnClientImpl {
 
     // Set java executable command 
     LOG.info("Setting up app master command");
-    vargs.add("${JAVA_HOME}" + "/bin/java");
+    vargs.add(Environment.JAVA_HOME.$() + "/bin/java");
     // Set Xmx based on am memory size
     vargs.add("-Xmx" + amMemory + "m");
     // Set class name 

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java

@@ -29,6 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.util.JarFinder;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -54,8 +55,8 @@ public class TestDistributedShell {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, 
         FifoScheduler.class, ResourceScheduler.class);
     if (yarnCluster == null) {
-      yarnCluster = new MiniYARNCluster(TestDistributedShell.class.getName(),
-          1, 1, 1);
+      yarnCluster = new MiniYARNCluster(
+        TestDistributedShell.class.getSimpleName(), 1, 1, 1);
       yarnCluster.init(conf);
       yarnCluster.start();
       URL url = Thread.currentThread().getContextClassLoader().getResource("yarn-site.xml");
@@ -82,7 +83,7 @@ public class TestDistributedShell {
     }
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testDSShell() throws Exception {
 
     String[] args = {
@@ -91,7 +92,7 @@ public class TestDistributedShell {
         "--num_containers",
         "2",
         "--shell_command",
-        "ls",
+        Shell.WINDOWS ? "dir" : "ls",
         "--master_memory",
         "512",
         "--container_memory",
@@ -110,7 +111,7 @@ public class TestDistributedShell {
 
   }
 
-  @Test
+  @Test(timeout=30000)
   public void testDSShellWithNoArgs() throws Exception {
 
     String[] args = {};

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -580,8 +580,10 @@ public class ContainerLaunch implements Callable<Integer> {
     // additional testing.  See YARN-358.
     if (Shell.WINDOWS) {
       String inputClassPath = environment.get(Environment.CLASSPATH.name());
-      environment.put(Environment.CLASSPATH.name(),
-          FileUtil.createJarWithClassPath(inputClassPath, pwd));
+      if (inputClassPath != null && !inputClassPath.isEmpty()) {
+        environment.put(Environment.CLASSPATH.name(),
+            FileUtil.createJarWithClassPath(inputClassPath, pwd));
+      }
     }
 
     /**

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestContainerLogsPage.java

@@ -41,11 +41,11 @@ import org.junit.Test;
 
 public class TestContainerLogsPage {
 
-  @Test
+  @Test(timeout=30000)
   public void testContainerLogDirs() throws IOException {
-    String absLogDir = new File("target",
-        TestNMWebServer.class.getSimpleName() + "LogDir").getAbsolutePath();
-    String logdirwithFile = "file://" + absLogDir;
+    File absLogDir = new File("target",
+      TestNMWebServer.class.getSimpleName() + "LogDir").getAbsoluteFile();
+    String logdirwithFile = absLogDir.toURI().toString();
     Configuration conf = new Configuration();
     conf.set(YarnConfiguration.NM_LOG_DIRS, logdirwithFile);
     NodeHealthCheckerService healthChecker = new NodeHealthCheckerService();

+ 7 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -405,10 +406,15 @@ public class TestContainerManagerSecurity {
       UnsupportedFileSystemException, YarnRemoteException,
       InterruptedException {
 
+    // Use ping to simulate sleep on Windows.
+    List<String> cmd = Shell.WINDOWS ?
+      Arrays.asList("ping", "-n", "100", "127.0.0.1", ">nul") :
+      Arrays.asList("sleep", "100");
+
     ContainerLaunchContext amContainer = BuilderUtils
         .newContainerLaunchContext("testUser",
             Collections.<String, LocalResource>emptyMap(),
-            new HashMap<String, String>(), Arrays.asList("sleep", "100"),
+            new HashMap<String, String>(), cmd,
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<ApplicationAccessType, String>());