浏览代码

HADOOP-10775. Shell operations to fail with meaningful errors on windows if winutils.exe not found. (stevel)

Steve Loughran 9 年之前
父节点
当前提交
5eab51a24f
共有 19 个文件被更改,包括 886 次插入365 次删除
  1. 3 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 0 29
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  3. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
  4. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java
  5. 18 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java
  6. 500 177
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java
  7. 3 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java
  8. 4 26
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/SymlinkBaseTest.java
  9. 1 7
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
  10. 0 11
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java
  11. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  12. 242 23
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
  13. 75 49
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java
  14. 7 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java
  15. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java
  16. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java
  17. 3 11
      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
  18. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerExecutor.java
  19. 15 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

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

@@ -300,6 +300,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12344. Improve validateSocketPathSecurity0 error message (Casey
     Brotherton via Colin P. McCabe)
 
+    HADOOP-10775. Shell operations to fail with meaningful errors on windows if
+    winutils.exe not found. (stevel)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

+ 0 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -780,35 +780,6 @@ public class FileUtil {
     File linkFile = new File(
         Path.getPathWithoutSchemeAndAuthority(new Path(linkname)).toString());
 
-    // If not on Java7+, copy a file instead of creating a symlink since
-    // Java6 has close to no support for symlinks on Windows. Specifically
-    // File#length and File#renameTo do not work as expected.
-    // (see HADOOP-9061 for additional details)
-    // We still create symlinks for directories, since the scenario in this
-    // case is different. The directory content could change in which
-    // case the symlink loses its purpose (for example task attempt log folder
-    // is symlinked under userlogs and userlogs are generated afterwards).
-    if (Shell.WINDOWS && !Shell.isJava7OrAbove() && targetFile.isFile()) {
-      try {
-        LOG.warn("FileUtil#symlink: On Windows+Java6, copying file instead " +
-            "of creating a symlink. Copying " + target + " -> " + linkname);
-
-        if (!linkFile.getParentFile().exists()) {
-          LOG.warn("Parent directory " + linkFile.getParent() +
-              " does not exist.");
-          return 1;
-        } else {
-          org.apache.commons.io.FileUtils.copyFile(targetFile, linkFile);
-        }
-      } catch (IOException ex) {
-        LOG.warn("FileUtil#symlink failed to copy the file with error: "
-            + ex.getMessage());
-        // Exit with non-zero exit code
-        return 1;
-      }
-      return 0;
-    }
-
     String[] cmd = Shell.getSymlinkCommand(
         targetFile.toString(),
         linkFile.toString());

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java

@@ -122,6 +122,12 @@ public class HardLink {
   @VisibleForTesting
   static class HardLinkCGWin extends HardLinkCommandGetter {
 
+    /**
+     * Build the windows link command. This must not
+     * use an exception-raising reference to WINUTILS, as
+     * some tests examine the command.
+     */
+    @SuppressWarnings("deprecation")
     static String[] getLinkCountCommand = {
         Shell.WINUTILS, "hardlink", "stat", null};
 
@@ -130,6 +136,8 @@ public class HardLink {
      */
     @Override
     String[] linkCount(File file) throws IOException {
+      // trigger the check for winutils
+      Shell.getWinutilsFile();
       String[] buf = new String[getLinkCountCommand.length];
       System.arraycopy(getLinkCountCommand, 0, buf, 0, 
                        getLinkCountCommand.length);

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java

@@ -75,7 +75,7 @@ public class DataChecksum implements Checksum {
    * is chosen depending on the platform.
    */
   public static Checksum newCrc32() {
-    return Shell.isJava7OrAbove()? new CRC32(): new PureJavaCrc32();
+    return new CRC32();
   }
 
   public static DataChecksum newDataChecksum(Type type, int bytesPerChecksum ) {

+ 18 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.util;
 
-import org.apache.hadoop.util.NativeCodeLoader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.OpensslCipher;
 import org.apache.hadoop.io.compress.Lz4Codec;
@@ -27,10 +26,17 @@ import org.apache.hadoop.io.compress.bzip2.Bzip2Factory;
 import org.apache.hadoop.io.compress.zlib.ZlibFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class NativeLibraryChecker {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(NativeLibraryChecker.class);
+
   /**
    * A tool to test native library availability, 
    */
@@ -99,12 +105,17 @@ public class NativeLibraryChecker {
       }
     }
 
-    // winutils.exe is required on Windows
-    winutilsPath = Shell.getWinUtilsPath();
-    if (winutilsPath != null) {
-      winutilsExists = true;
-    } else {
-      winutilsPath = "";
+    if (Shell.WINDOWS) {
+      // winutils.exe is required on Windows
+      try {
+        winutilsPath = Shell.getWinutilsFile().getCanonicalPath();
+        winutilsExists = true;
+      } catch (IOException e) {
+        LOG.debug("No Winutils: ", e);
+        winutilsPath = e.getMessage();
+        winutilsExists = false;
+      }
+      System.out.printf("winutils: %b %s%n", winutilsExists, winutilsPath);
     }
 
     System.out.println("Native library checking:");

文件差异内容过多而无法显示
+ 500 - 177
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java


+ 3 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SysInfoWindows.java

@@ -70,9 +70,10 @@ public class SysInfoWindows extends SysInfo {
   }
 
   String getSystemInfoInfoFromShell() {
-    ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
-        new String[] {Shell.WINUTILS, "systeminfo" });
     try {
+      ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
+          new String[] {Shell.getWinutilsFile().getCanonicalPath(),
+              "systeminfo" });
       shellExecutor.execute();
       return shellExecutor.getOutput();
     } catch (IOException e) {

+ 4 - 26
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/SymlinkBaseTest.java

@@ -52,13 +52,6 @@ public abstract class SymlinkBaseTest {
   abstract protected String testBaseDir2() throws IOException;
   abstract protected URI testURI();
 
-  // Returns true if the filesystem is emulating symlink support. Certain
-  // checks will be bypassed if that is the case.
-  //
-  protected boolean emulatingSymlinksOnWindows() {
-    return false;
-  }
-
   protected IOException unwrapException(IOException e) {
     return e;
   }
@@ -235,7 +228,6 @@ public abstract class SymlinkBaseTest {
   @Test(timeout=10000)
   /** Stat a link to a file */
   public void testStatLinkToFile() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path file = new Path(testBaseDir1()+"/file");
     Path linkToFile = new Path(testBaseDir1()+"/linkToFile");
     createAndWriteFile(file);
@@ -362,11 +354,6 @@ public abstract class SymlinkBaseTest {
   private void checkLink(Path linkAbs, Path expectedTarget, Path targetQual)
       throws IOException {
 
-    // If we are emulating symlinks then many of these checks will fail
-    // so we skip them.
-    //
-    assumeTrue(!emulatingSymlinksOnWindows());
-
     Path dir = new Path(testBaseDir1());
     // isFile/Directory
     assertTrue(wrapper.isFile(linkAbs));
@@ -663,7 +650,6 @@ public abstract class SymlinkBaseTest {
   @Test(timeout=10000)
   /** Create symlink through a symlink */
   public void testCreateLinkViaLink() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path dir1        = new Path(testBaseDir1());
     Path file        = new Path(testBaseDir1(), "file");
     Path linkToDir   = new Path(testBaseDir2(), "linkToDir");
@@ -706,7 +692,6 @@ public abstract class SymlinkBaseTest {
   @Test(timeout=10000)
   /** Test create symlink using the same path */
   public void testCreateLinkTwice() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path file = new Path(testBaseDir1(), "file");
     Path link = new Path(testBaseDir1(), "linkToFile");
     createAndWriteFile(file);
@@ -895,8 +880,7 @@ public abstract class SymlinkBaseTest {
     assertFalse(wrapper.exists(linkViaLink));
     // Check that we didn't rename the link target
     assertTrue(wrapper.exists(file));
-    assertTrue(wrapper.getFileLinkStatus(linkNewViaLink).isSymlink() ||
-        emulatingSymlinksOnWindows());
+    assertTrue(wrapper.getFileLinkStatus(linkNewViaLink).isSymlink());
     readFile(linkNewViaLink);
   }
 
@@ -1034,8 +1018,7 @@ public abstract class SymlinkBaseTest {
     createAndWriteFile(file);
     wrapper.createSymlink(file, link1, false);
     wrapper.rename(link1, link2);
-    assertTrue(wrapper.getFileLinkStatus(link2).isSymlink() ||
-        emulatingSymlinksOnWindows());
+    assertTrue(wrapper.getFileLinkStatus(link2).isSymlink());
     readFile(link2);
     readFile(file);
     assertFalse(wrapper.exists(link1));
@@ -1059,11 +1042,8 @@ public abstract class SymlinkBaseTest {
     }
     wrapper.rename(link, file1, Rename.OVERWRITE);
     assertFalse(wrapper.exists(link));
-
-    if (!emulatingSymlinksOnWindows()) {
-      assertTrue(wrapper.getFileLinkStatus(file1).isSymlink());
-      assertEquals(file2, wrapper.getLinkTarget(file1));
-    }
+    assertTrue(wrapper.getFileLinkStatus(file1).isSymlink());
+    assertEquals(file2, wrapper.getLinkTarget(file1));
   }
 
   @Test(timeout=10000)
@@ -1125,7 +1105,6 @@ public abstract class SymlinkBaseTest {
   @Test(timeout=10000)
   /** Rename a symlink */
   public void testRenameSymlink() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path file  = new Path(testBaseDir1(), "file");
     Path link1 = new Path(testBaseDir1(), "linkToFile1");
     Path link2 = new Path(testBaseDir1(), "linkToFile2");
@@ -1223,7 +1202,6 @@ public abstract class SymlinkBaseTest {
   @Test(timeout=10000)
   /** Test rename the symlink's target */
   public void testRenameLinkTarget() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path file    = new Path(testBaseDir1(), "file");
     Path fileNew = new Path(testBaseDir1(), "fileNew");
     Path link    = new Path(testBaseDir1(), "linkToFile");

+ 1 - 7
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java

@@ -983,13 +983,7 @@ public class TestFileUtil {
     file.delete();
     Assert.assertFalse(file.exists());
 
-    if (Shell.WINDOWS && !Shell.isJava7OrAbove()) {
-      // On Java6 on Windows, we copied the file
-      Assert.assertEquals(data.length, link.length());
-    } else {
-      // Otherwise, the target file size is zero
-      Assert.assertEquals(0, link.length());
-    }
+    Assert.assertEquals(0, link.length());
 
     link.delete();
     Assert.assertFalse(link.exists());

+ 0 - 11
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

@@ -68,16 +68,6 @@ abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
     }
   }
 
-  @Override
-  protected boolean emulatingSymlinksOnWindows() {
-    // Java 6 on Windows has very poor symlink support. Specifically
-    // Specifically File#length and File#renameTo do not work as expected.
-    // (see HADOOP-9061 for additional details)
-    // Hence some symlink tests will be skipped.
-    //
-    return (Shell.WINDOWS && !Shell.isJava7OrAbove());
-  }
-
   @Override
   public void testCreateDanglingLink() throws IOException {
     // Dangling symlinks are not supported on Windows local file system.
@@ -186,7 +176,6 @@ abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
    * file scheme (eg file://host/tmp/test).
    */  
   public void testGetLinkStatusPartQualTarget() throws IOException {
-    assumeTrue(!emulatingSymlinksOnWindows());
     Path fileAbs  = new Path(testBaseDir1()+"/file");
     Path fileQual = new Path(testURI().toString(), fileAbs);
     Path dir      = new Path(testBaseDir1());

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java

@@ -217,7 +217,8 @@ public class TestUserGroupInformation {
     }
     // get the groups
     pp = Runtime.getRuntime().exec(Shell.WINDOWS ?
-      Shell.WINUTILS + " groups -F" : "id -Gn");
+      Shell.getWinutilsPath() + " groups -F"
+      : "id -Gn");
     br = new BufferedReader(new InputStreamReader(pp.getInputStream()));
     String line = br.readLine();
 

+ 242 - 23
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java

@@ -17,11 +17,12 @@
  */
 package org.apache.hadoop.util;
 
-import junit.framework.TestCase;
+import org.apache.commons.io.FileUtils;
 import org.junit.Assert;
 
 import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -30,8 +31,31 @@ import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 
 import org.apache.hadoop.fs.FileUtil;
+import static org.apache.hadoop.util.Shell.*;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.junit.rules.Timeout;
 
-public class TestShell extends TestCase {
+public class TestShell extends Assert {
+  /**
+   * Set the timeout for every test
+   */
+  @Rule
+  public Timeout testTimeout = new Timeout(30000);
+
+  @Rule
+  public TestName methodName = new TestName();
+
+  private File rootTestDir = new File(System.getProperty("test.build.data", "target/"));
+
+  /**
+   * A filename generated uniquely for each test method. The file
+   * itself is neither created nor deleted during test setup/teardown.
+   */
+  private File methodDir;
 
   private static class Command extends Shell {
     private int runCount = 0;
@@ -45,7 +69,7 @@ public class TestShell extends TestCase {
       // There is no /bin/echo equivalent on Windows so just launch it as a
       // shell built-in.
       //
-      return Shell.WINDOWS ?
+      return WINDOWS ?
           (new String[] {"cmd.exe", "/c", "echo", "hello"}) :
           (new String[] {"echo", "hello"});
     }
@@ -60,6 +84,14 @@ public class TestShell extends TestCase {
     }
   }
 
+  @Before
+  public void setup() {
+    rootTestDir.mkdirs();
+    assertTrue("Not a directory " + rootTestDir, rootTestDir.isDirectory());
+    methodDir = new File(rootTestDir, methodName.getMethodName());
+  }
+
+  @Test
   public void testInterval() throws IOException {
     testInterval(Long.MIN_VALUE / 60000);  // test a negative interval
     testInterval(0L);  // test a zero interval
@@ -79,6 +111,7 @@ public class TestShell extends TestCase {
     }
   }
 
+  @Test
   public void testShellCommandExecutorToString() throws Throwable {
     Shell.ShellCommandExecutor sce=new Shell.ShellCommandExecutor(
             new String[] { "ls", "..","arg 2"});
@@ -87,30 +120,28 @@ public class TestShell extends TestCase {
     assertInString(command, " .. ");
     assertInString(command, "\"arg 2\"");
   }
-  
+
+  @Test
   public void testShellCommandTimeout() throws Throwable {
-    if(Shell.WINDOWS) {
-      // setExecutable does not work on Windows
-      return;
-    }
-    String rootDir = new File(System.getProperty(
-        "test.build.data", "/tmp")).getAbsolutePath();
+    Assume.assumeFalse(WINDOWS);
+    String rootDir = rootTestDir.getAbsolutePath();
     File shellFile = new File(rootDir, "timeout.sh");
     String timeoutCommand = "sleep 4; echo \"hello\"";
-    PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
-    writer.println(timeoutCommand);
-    writer.close();
+    Shell.ShellCommandExecutor shexc;
+    try (PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile))) {
+      writer.println(timeoutCommand);
+      writer.close();
+    }
     FileUtil.setExecutable(shellFile, true);
-    Shell.ShellCommandExecutor shexc 
-    = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()},
-                                      null, null, 100);
+    shexc = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()},
+        null, null, 100);
     try {
       shexc.execute();
     } catch (Exception e) {
       //When timing out exception is thrown.
     }
     shellFile.delete();
-    assertTrue("Script didnt not timeout" , shexc.isTimedOut());
+    assertTrue("Script did not timeout" , shexc.isTimedOut());
   }
   
   private static int countTimerThreads() {
@@ -129,7 +160,8 @@ public class TestShell extends TestCase {
     }
     return count;
   }
-  
+
+  @Test
   public void testShellCommandTimerLeak() throws Exception {
     String quickCommand[] = new String[] {"/bin/sleep", "100"};
     
@@ -152,16 +184,17 @@ public class TestShell extends TestCase {
     assertEquals(timersBefore, timersAfter);
   }
 
+  @Test
   public void testGetCheckProcessIsAliveCommand() throws Exception {
     String anyPid = "9999";
-    String[] checkProcessAliveCommand = Shell.getCheckProcessIsAliveCommand(
+    String[] checkProcessAliveCommand = getCheckProcessIsAliveCommand(
         anyPid);
 
     String[] expectedCommand;
 
     if (Shell.WINDOWS) {
       expectedCommand =
-          new String[]{ Shell.WINUTILS, "task", "isAlive", anyPid };
+          new String[]{getWinutilsPath(), "task", "isAlive", anyPid };
     } else if (Shell.isSetsidAvailable) {
       expectedCommand = new String[] { "bash", "-c", "kill -0 -- -" + anyPid };
     } else {
@@ -170,17 +203,18 @@ public class TestShell extends TestCase {
     Assert.assertArrayEquals(expectedCommand, checkProcessAliveCommand);
   }
 
+  @Test
   public void testGetSignalKillCommand() throws Exception {
     String anyPid = "9999";
     int anySignal = 9;
-    String[] checkProcessAliveCommand = Shell.getSignalKillCommand(anySignal,
+    String[] checkProcessAliveCommand = getSignalKillCommand(anySignal,
         anyPid);
 
     String[] expectedCommand;
 
     if (Shell.WINDOWS) {
       expectedCommand =
-          new String[]{ Shell.WINUTILS, "task", "isAlive", anyPid };
+          new String[]{getWinutilsPath(), "task", "kill", anyPid };
     } else if (Shell.isSetsidAvailable) {
       expectedCommand = new String[] { "bash", "-c", "kill -9 -- -" + anyPid };
     } else {
@@ -188,7 +222,6 @@ public class TestShell extends TestCase {
     }
     Assert.assertArrayEquals(expectedCommand, checkProcessAliveCommand);
   }
-  
 
   private void testInterval(long interval) throws IOException {
     Command command = new Command(interval);
@@ -203,4 +236,190 @@ public class TestShell extends TestCase {
       assertEquals(2, command.getRunCount());
     }
   }
+
+  @Test
+  public void testHadoopHomeUnset() throws Throwable {
+    assertHomeResolveFailed(null, "unset");
+  }
+
+  @Test
+  public void testHadoopHomeEmpty() throws Throwable {
+    assertHomeResolveFailed("", E_HADOOP_PROPS_EMPTY);
+  }
+
+  @Test
+  public void testHadoopHomeEmptyDoubleQuotes() throws Throwable {
+    assertHomeResolveFailed("\"\"", E_HADOOP_PROPS_EMPTY);
+  }
+
+  @Test
+  public void testHadoopHomeEmptySingleQuote() throws Throwable {
+    assertHomeResolveFailed("\"", E_HADOOP_PROPS_EMPTY);
+  }
+
+  @Test
+  public void testHadoopHomeValid() throws Throwable {
+    File f = checkHadoopHomeInner(rootTestDir.getCanonicalPath());
+    assertEquals(rootTestDir, f);
+  }
+
+  @Test
+  public void testHadoopHomeValidQuoted() throws Throwable {
+    File f = checkHadoopHomeInner('"'+ rootTestDir.getCanonicalPath() + '"');
+    assertEquals(rootTestDir, f);
+  }
+
+  @Test
+  public void testHadoopHomeNoDir() throws Throwable {
+    assertHomeResolveFailed(methodDir.getCanonicalPath(), E_DOES_NOT_EXIST);
+  }
+
+  @Test
+  public void testHadoopHomeNotADir() throws Throwable {
+    File touched = touch(methodDir);
+    try {
+      assertHomeResolveFailed(touched.getCanonicalPath(), E_NOT_DIRECTORY);
+    } finally {
+      FileUtils.deleteQuietly(touched);
+    }
+  }
+
+  @Test
+  public void testHadoopHomeRelative() throws Throwable {
+    assertHomeResolveFailed("./target", E_IS_RELATIVE);
+  }
+
+  @Test
+  public void testBinDirMissing() throws Throwable {
+    FileNotFoundException ex = assertWinutilsResolveFailed(methodDir,
+        E_DOES_NOT_EXIST);
+    assertInString(ex.toString(), "Hadoop bin directory");
+  }
+
+  @Test
+  public void testHadoopBinNotADir() throws Throwable {
+    File bin = new File(methodDir, "bin");
+    touch(bin);
+    try {
+      assertWinutilsResolveFailed(methodDir, E_NOT_DIRECTORY);
+    } finally {
+      FileUtils.deleteQuietly(methodDir);
+    }
+  }
+
+  @Test
+  public void testBinWinUtilsFound() throws Throwable {
+    try {
+      File bin = new File(methodDir, "bin");
+      File winutils = new File(bin, WINUTILS_EXE);
+      touch(winutils);
+      assertEquals(winutils.getCanonicalPath(),
+          getQualifiedBinInner(methodDir, WINUTILS_EXE).getCanonicalPath());
+    } finally {
+      FileUtils.deleteQuietly(methodDir);
+    }
+  }
+
+  @Test
+  public void testBinWinUtilsNotAFile() throws Throwable {
+    try {
+      File bin = new File(methodDir, "bin");
+      File winutils = new File(bin, WINUTILS_EXE);
+      winutils.mkdirs();
+      assertWinutilsResolveFailed(methodDir, E_NOT_EXECUTABLE_FILE);
+    } finally {
+      FileUtils.deleteDirectory(methodDir);
+    }
+  }
+
+  /**
+   * This test takes advantage of the invariant winutils path is valid
+   * or access to it will raise an exception holds on Linux, and without
+   * any winutils binary even if HADOOP_HOME points to a real hadoop
+   * directory, the exception reporting can be validated
+   */
+  @Test
+  public void testNoWinutilsOnUnix() throws Throwable {
+    Assume.assumeFalse(WINDOWS);
+    try {
+      getWinutilsFile();
+    } catch (FileNotFoundException ex) {
+      assertExContains(ex, E_NOT_A_WINDOWS_SYSTEM);
+    }
+    try {
+      getWinutilsPath();
+    } catch (RuntimeException ex) {
+      assertExContains(ex, E_NOT_A_WINDOWS_SYSTEM);
+      if ( ex.getCause() == null
+          || !(ex.getCause() instanceof FileNotFoundException)) {
+        throw ex;
+      }
+    }
+  }
+
+  /**
+   * Touch a file; creating parent dirs on demand.
+   * @param path path of file
+   * @return the file created
+   * @throws IOException on any failure to write
+   */
+  private File touch(File path) throws IOException {
+    path.getParentFile().mkdirs();
+    FileUtils.writeByteArrayToFile(path, new byte[]{});
+    return path;
+  }
+
+  /**
+   * Assert that an attept to resolve the hadoop home dir failed with
+   * an expected text in the exception string value.
+   * @param path input
+   * @param expectedText expected exception text
+   * @return the caught exception
+   * @throws FileNotFoundException any FileNotFoundException that was thrown
+   * but which did not contain the expected text
+   */
+  private FileNotFoundException assertHomeResolveFailed(String path,
+      String expectedText) throws Exception {
+    try {
+      File f = checkHadoopHomeInner(path);
+      fail("Expected an exception with the text `" + expectedText + "`"
+          + " -but got the path " + f);
+      // unreachable
+      return null;
+    } catch (FileNotFoundException ex) {
+      assertExContains(ex, expectedText);
+      return ex;
+    }
+  }
+
+  /**
+   * Assert that an attept to resolve the {@code bin/winutils.exe} failed with
+   * an expected text in the exception string value.
+   * @param hadoopHome hadoop home directory
+   * @param expectedText expected exception text
+   * @return the caught exception
+   * @throws Exception any Exception that was thrown
+   * but which did not contain the expected text
+   */
+  private FileNotFoundException assertWinutilsResolveFailed(File hadoopHome,
+      String expectedText) throws Exception {
+    try {
+      File f = getQualifiedBinInner(hadoopHome, WINUTILS_EXE);
+      fail("Expected an exception with the text `" + expectedText + "`"
+          + " -but got the path " + f);
+      // unreachable
+      return null;
+    } catch (FileNotFoundException ex) {
+      assertExContains(ex, expectedText);
+      return ex;
+    }
+  }
+
+  private void assertExContains(Exception ex, String expectedText)
+      throws Exception {
+    if (!ex.toString().contains(expectedText)) {
+      throw ex;
+    }
+  }
+
 }

+ 75 - 49
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestWinUtils.java

@@ -45,13 +45,18 @@ public class TestWinUtils {
 
   private static final Log LOG = LogFactory.getLog(TestWinUtils.class);
   private static File TEST_DIR = new File(System.getProperty("test.build.data",
-      "/tmp"), TestWinUtils.class.getSimpleName());
+      "target"+File.pathSeparator + "tmp"), TestWinUtils.class.getSimpleName());
+
+  String winutils;
 
   @Before
-  public void setUp() {
+  public void setUp() throws IOException {
     // Not supported on non-Windows platforms
     assumeTrue(Shell.WINDOWS);
     TEST_DIR.mkdirs();
+    assertTrue("Failed to create Test directory " + TEST_DIR,
+        TEST_DIR.isDirectory() );
+    winutils = Shell.getWinutilsPath();
   }
 
   @After
@@ -59,46 +64,55 @@ public class TestWinUtils {
     FileUtil.fullyDelete(TEST_DIR);
   }
 
+  private void requireWinutils() throws IOException {
+    Shell.getWinutilsPath();
+  }
+
   // Helper routine that writes the given content to the file.
   private void writeFile(File file, String content) throws IOException {
     byte[] data = content.getBytes();
-    FileOutputStream os = new FileOutputStream(file);
-    os.write(data);
-    os.close();
+    try (FileOutputStream os = new FileOutputStream(file)) {
+      os.write(data);
+      os.close();
+    }
   }
 
   // Helper routine that reads the first 100 bytes from the file.
   private String readFile(File file) throws IOException {
-    FileInputStream fos = new FileInputStream(file);
-    byte[] b = new byte[100];
-    fos.read(b);
-    return b.toString();
+    byte[] b;
+    try (FileInputStream fos = new FileInputStream(file)) {
+      b = new byte[100];
+      int count = fos.read(b);
+      assertEquals(100, count);
+    }
+    return new String(b);
   }
 
   @Test (timeout = 30000)
   public void testLs() throws IOException {
+    requireWinutils();
     final String content = "6bytes";
     final int contentSize = content.length();
     File testFile = new File(TEST_DIR, "file1");
     writeFile(testFile, content);
 
     // Verify permissions and file name return tokens
+    String testPath = testFile.getCanonicalPath();
     String output = Shell.execCommand(
-        Shell.WINUTILS, "ls", testFile.getCanonicalPath());
+        winutils, "ls", testPath);
     String[] outputArgs = output.split("[ \r\n]");
-    assertTrue(outputArgs[0].equals("-rwx------"));
-    assertTrue(outputArgs[outputArgs.length - 1]
-        .equals(testFile.getCanonicalPath()));
+    assertEquals("-rwx------", outputArgs[0]);
+    assertEquals(outputArgs[outputArgs.length - 1], testPath);
 
     // Verify most tokens when using a formatted output (other tokens
     // will be verified with chmod/chown)
     output = Shell.execCommand(
-        Shell.WINUTILS, "ls", "-F", testFile.getCanonicalPath());
+        winutils, "ls", "-F", testPath);
     outputArgs = output.split("[|\r\n]");
     assertEquals(9, outputArgs.length);
-    assertTrue(outputArgs[0].equals("-rwx------"));
+    assertEquals("-rwx------", outputArgs[0]);
     assertEquals(contentSize, Long.parseLong(outputArgs[4]));
-    assertTrue(outputArgs[8].equals(testFile.getCanonicalPath()));
+    assertEquals(outputArgs[8], testPath);
 
     testFile.delete();
     assertFalse(testFile.exists());
@@ -106,41 +120,42 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testGroups() throws IOException {
+    requireWinutils();
     String currentUser = System.getProperty("user.name");
 
     // Verify that groups command returns information about the current user
     // groups when invoked with no args
     String outputNoArgs = Shell.execCommand(
-        Shell.WINUTILS, "groups").trim();
+        winutils, "groups").trim();
     String output = Shell.execCommand(
-        Shell.WINUTILS, "groups", currentUser).trim();
+        winutils, "groups", currentUser).trim();
     assertEquals(output, outputNoArgs);
 
     // Verify that groups command with the -F flag returns the same information
     String outputFormat = Shell.execCommand(
-        Shell.WINUTILS, "groups", "-F", currentUser).trim();
+        winutils, "groups", "-F", currentUser).trim();
     outputFormat = outputFormat.replace("|", " ");
     assertEquals(output, outputFormat);
   }
 
   private void chmod(String mask, File file) throws IOException {
     Shell.execCommand(
-        Shell.WINUTILS, "chmod", mask, file.getCanonicalPath());
+        winutils, "chmod", mask, file.getCanonicalPath());
   }
 
   private void chmodR(String mask, File file) throws IOException {
     Shell.execCommand(
-        Shell.WINUTILS, "chmod", "-R", mask, file.getCanonicalPath());
+        winutils, "chmod", "-R", mask, file.getCanonicalPath());
   }
 
   private String ls(File file) throws IOException {
     return Shell.execCommand(
-        Shell.WINUTILS, "ls", file.getCanonicalPath());
+        winutils, "ls", file.getCanonicalPath());
   }
 
   private String lsF(File file) throws IOException {
     return Shell.execCommand(
-        Shell.WINUTILS, "ls", "-F", file.getCanonicalPath());
+        winutils, "ls", "-F", file.getCanonicalPath());
   }
 
   private void assertPermissions(File file, String expected)
@@ -151,6 +166,7 @@ public class TestWinUtils {
 
   private void testChmodInternal(String mode, String expectedPerm)
       throws IOException {
+    requireWinutils();
     File a = new File(TEST_DIR, "file1");
     assertTrue(a.createNewFile());
 
@@ -168,6 +184,7 @@ public class TestWinUtils {
   }
 
   private void testNewFileChmodInternal(String expectedPerm) throws IOException {
+    requireWinutils();
     // Create a new directory
     File dir = new File(TEST_DIR, "dir1");
 
@@ -190,6 +207,7 @@ public class TestWinUtils {
 
   private void testChmodInternalR(String mode, String expectedPerm,
       String expectedPermx) throws IOException {
+    requireWinutils();
     // Setup test folder hierarchy
     File a = new File(TEST_DIR, "a");
     assertTrue(a.mkdir());
@@ -226,6 +244,7 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testBasicChmod() throws IOException {
+    requireWinutils();
     // - Create a file.
     // - Change mode to 377 so owner does not have read permission.
     // - Verify the owner truly does not have the permissions to read.
@@ -249,7 +268,7 @@ public class TestWinUtils {
  
     try {
       writeFile(a, "test");
-      assertFalse("writeFile should have failed!", true);
+      fail("writeFile should have failed!");
     } catch (IOException ex) {
       LOG.info("Expected: Failed write to a file with permissions 577");
     }
@@ -261,14 +280,14 @@ public class TestWinUtils {
     // - Change mode to 677 so owner does not have execute permission.
     // - Verify the owner truly does not have the permissions to execute the file.
 
-    File winutilsFile = new File(Shell.WINUTILS);
+    File winutilsFile = Shell.getWinutilsFile();
     File aExe = new File(TEST_DIR, "a.exe");
     FileUtils.copyFile(winutilsFile, aExe);
     chmod("677", aExe);
 
     try {
       Shell.execCommand(aExe.getCanonicalPath(), "ls");
-      assertFalse("executing " + aExe + " should have failed!", true);
+      fail("executing " + aExe + " should have failed!");
     } catch (IOException ex) {
       LOG.info("Expected: Failed to execute a file with permissions 677");
     }
@@ -278,6 +297,7 @@ public class TestWinUtils {
   /** Validate behavior of chmod commands on directories on Windows. */
   @Test (timeout = 30000)
   public void testBasicChmodOnDir() throws IOException {
+    requireWinutils();
     // Validate that listing a directory with no read permission fails
     File a = new File(TEST_DIR, "a");
     File b = new File(a, "b");
@@ -287,8 +307,7 @@ public class TestWinUtils {
     // Remove read permissions on directory a
     chmod("300", a);
     String[] files = a.list();
-    assertTrue("Listing a directory without read permission should fail",
-        null == files);
+    assertNull("Listing a directory without read permission should fail", files);
 
     // restore permissions
     chmod("700", a);
@@ -306,7 +325,7 @@ public class TestWinUtils {
       // FILE_WRITE_DATA/FILE_ADD_FILE privilege is denied on
       // the dir.
       c.createNewFile();
-      assertFalse("writeFile should have failed!", true);
+      fail("writeFile should have failed!");
     } catch (IOException ex) {
       LOG.info("Expected: Failed to create a file when directory "
           + "permissions are 577");
@@ -356,6 +375,7 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChmod() throws IOException {
+    requireWinutils();
     testChmodInternal("7", "-------rwx");
     testChmodInternal("70", "----rwx---");
     testChmodInternal("u-x,g+r,o=g", "-rw-r--r--");
@@ -376,7 +396,7 @@ public class TestWinUtils {
 
   private void chown(String userGroup, File file) throws IOException {
     Shell.execCommand(
-        Shell.WINUTILS, "chown", userGroup, file.getCanonicalPath());
+        winutils, "chown", userGroup, file.getCanonicalPath());
   }
 
   private void assertOwners(File file, String expectedUser,
@@ -390,6 +410,7 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testChown() throws IOException {
+    requireWinutils();
     File a = new File(TEST_DIR, "a");
     assertTrue(a.createNewFile());
     String username = System.getProperty("user.name");
@@ -415,12 +436,13 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testSymlinkRejectsForwardSlashesInLink() throws IOException {
+    requireWinutils();
     File newFile = new File(TEST_DIR, "file");
     assertTrue(newFile.createNewFile());
     String target = newFile.getPath();
     String link = new File(TEST_DIR, "link").getPath().replaceAll("\\\\", "/");
     try {
-      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      Shell.execCommand(winutils, "symlink", link, target);
       fail(String.format("did not receive expected failure creating symlink "
         + "with forward slashes in link: link = %s, target = %s", link, target));
     } catch (IOException e) {
@@ -431,12 +453,13 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testSymlinkRejectsForwardSlashesInTarget() throws IOException {
+    requireWinutils();
     File newFile = new File(TEST_DIR, "file");
     assertTrue(newFile.createNewFile());
     String target = newFile.getPath().replaceAll("\\\\", "/");
     String link = new File(TEST_DIR, "link").getPath();
     try {
-      Shell.execCommand(Shell.WINUTILS, "symlink", link, target);
+      Shell.execCommand(winutils, "symlink", link, target);
       fail(String.format("did not receive expected failure creating symlink "
         + "with forward slashes in target: link = %s, target = %s", link, target));
     } catch (IOException e) {
@@ -447,6 +470,7 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testReadLink() throws IOException {
+    requireWinutils();
     // Create TEST_DIR\dir1\file1.txt
     //
     File dir1 = new File(TEST_DIR, "dir1");
@@ -462,18 +486,18 @@ public class TestWinUtils {
     // symlink to file1.txt.
     //
     Shell.execCommand(
-        Shell.WINUTILS, "symlink", dirLink.toString(), dir1.toString());
+        winutils, "symlink", dirLink.toString(), dir1.toString());
     Shell.execCommand(
-        Shell.WINUTILS, "symlink", fileLink.toString(), file1.toString());
+        winutils, "symlink", fileLink.toString(), file1.toString());
 
     // Read back the two links and ensure we get what we expected.
     //
-    String readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+    String readLinkOutput = Shell.execCommand(winutils,
         "readlink",
         dirLink.toString());
     assertThat(readLinkOutput, equalTo(dir1.toString()));
 
-    readLinkOutput = Shell.execCommand(Shell.WINUTILS,
+    readLinkOutput = Shell.execCommand(winutils,
         "readlink",
         fileLink.toString());
     assertThat(readLinkOutput, equalTo(file1.toString()));
@@ -483,7 +507,7 @@ public class TestWinUtils {
     try {
       // No link name specified.
       //
-      Shell.execCommand(Shell.WINUTILS, "readlink", "");
+      Shell.execCommand(winutils, "readlink", "");
       fail("Failed to get Shell.ExitCodeException when reading bad symlink");
     } catch (Shell.ExitCodeException ece) {
       assertThat(ece.getExitCode(), is(1));
@@ -492,7 +516,7 @@ public class TestWinUtils {
     try {
       // Bad link name.
       //
-      Shell.execCommand(Shell.WINUTILS, "readlink", "ThereIsNoSuchLink");
+      Shell.execCommand(winutils, "readlink", "ThereIsNoSuchLink");
       fail("Failed to get Shell.ExitCodeException when reading bad symlink");
     } catch (Shell.ExitCodeException ece) {
       assertThat(ece.getExitCode(), is(1));
@@ -501,7 +525,7 @@ public class TestWinUtils {
     try {
       // Non-symlink directory target.
       //
-      Shell.execCommand(Shell.WINUTILS, "readlink", dir1.toString());
+      Shell.execCommand(winutils, "readlink", dir1.toString());
       fail("Failed to get Shell.ExitCodeException when reading bad symlink");
     } catch (Shell.ExitCodeException ece) {
       assertThat(ece.getExitCode(), is(1));
@@ -510,7 +534,7 @@ public class TestWinUtils {
     try {
       // Non-symlink file target.
       //
-      Shell.execCommand(Shell.WINUTILS, "readlink", file1.toString());
+      Shell.execCommand(winutils, "readlink", file1.toString());
       fail("Failed to get Shell.ExitCodeException when reading bad symlink");
     } catch (Shell.ExitCodeException ece) {
       assertThat(ece.getExitCode(), is(1));
@@ -519,7 +543,7 @@ public class TestWinUtils {
     try {
       // Too many parameters.
       //
-      Shell.execCommand(Shell.WINUTILS, "readlink", "a", "b");
+      Shell.execCommand(winutils, "readlink", "a", "b");
       fail("Failed to get Shell.ExitCodeException with bad parameters");
     } catch (Shell.ExitCodeException ece) {
       assertThat(ece.getExitCode(), is(1));
@@ -529,6 +553,7 @@ public class TestWinUtils {
   @SuppressWarnings("deprecation")
   @Test(timeout=10000)
   public void testTaskCreate() throws IOException {
+    requireWinutils();
     File batch = new File(TEST_DIR, "testTaskCreate.cmd");
     File proof = new File(TEST_DIR, "testTaskCreate.out");
     FileWriter fw = new FileWriter(batch);
@@ -538,7 +563,7 @@ public class TestWinUtils {
     
     assertFalse(proof.exists());
     
-    Shell.execCommand(Shell.WINUTILS, "task", "create", "testTaskCreate" + testNumber, 
+    Shell.execCommand(winutils, "task", "create", "testTaskCreate" + testNumber,
         batch.getAbsolutePath());
     
     assertTrue(proof.exists());
@@ -550,30 +575,31 @@ public class TestWinUtils {
 
   @Test (timeout = 30000)
   public void testTaskCreateWithLimits() throws IOException {
+    requireWinutils();
     // Generate a unique job id
     String jobId = String.format("%f", Math.random());
 
     // Run a task without any options
-    String out = Shell.execCommand(Shell.WINUTILS, "task", "create",
+    String out = Shell.execCommand(winutils, "task", "create",
         "job" + jobId, "cmd /c echo job" + jobId);
     assertTrue(out.trim().equals("job" + jobId));
 
     // Run a task without any limits
     jobId = String.format("%f", Math.random());
-    out = Shell.execCommand(Shell.WINUTILS, "task", "create", "-c", "-1", "-m",
+    out = Shell.execCommand(winutils, "task", "create", "-c", "-1", "-m",
         "-1", "job" + jobId, "cmd /c echo job" + jobId);
     assertTrue(out.trim().equals("job" + jobId));
 
     // Run a task with limits (128MB should be enough for a cmd)
     jobId = String.format("%f", Math.random());
-    out = Shell.execCommand(Shell.WINUTILS, "task", "create", "-c", "10000", "-m",
+    out = Shell.execCommand(winutils, "task", "create", "-c", "10000", "-m",
         "128", "job" + jobId, "cmd /c echo job" + jobId);
     assertTrue(out.trim().equals("job" + jobId));
 
     // Run a task without enough memory
     try {
       jobId = String.format("%f", Math.random());
-      out = Shell.execCommand(Shell.WINUTILS, "task", "create", "-m", "128", "job"
+      out = Shell.execCommand(winutils, "task", "create", "-m", "128", "job"
           + jobId, "java -Xmx256m -version");
       fail("Failed to get Shell.ExitCodeException with insufficient memory");
     } catch (Shell.ExitCodeException ece) {
@@ -584,7 +610,7 @@ public class TestWinUtils {
     //
     try {
       jobId = String.format("%f", Math.random());
-      Shell.execCommand(Shell.WINUTILS, "task", "create", "-c", "-1", "-m",
+      Shell.execCommand(winutils, "task", "create", "-c", "-1", "-m",
           "-1", "foo", "job" + jobId, "cmd /c echo job" + jobId);
       fail("Failed to get Shell.ExitCodeException with bad parameters");
     } catch (Shell.ExitCodeException ece) {
@@ -593,7 +619,7 @@ public class TestWinUtils {
 
     try {
       jobId = String.format("%f", Math.random());
-      Shell.execCommand(Shell.WINUTILS, "task", "create", "-c", "-m", "-1",
+      Shell.execCommand(winutils, "task", "create", "-c", "-m", "-1",
           "job" + jobId, "cmd /c echo job" + jobId);
       fail("Failed to get Shell.ExitCodeException with bad parameters");
     } catch (Shell.ExitCodeException ece) {
@@ -602,7 +628,7 @@ public class TestWinUtils {
 
     try {
       jobId = String.format("%f", Math.random());
-      Shell.execCommand(Shell.WINUTILS, "task", "create", "-c", "foo",
+      Shell.execCommand(winutils, "task", "create", "-c", "foo",
           "job" + jobId, "cmd /c echo job" + jobId);
       fail("Failed to get Shell.ExitCodeException with bad parameters");
     } catch (Shell.ExitCodeException ece) {

+ 7 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsBasedProcessTree.java

@@ -51,8 +51,11 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
     
   public static boolean isAvailable() {
     if (Shell.WINDOWS) {
+      if (!Shell.hasWinutilsPath()) {
+        return false;
+      }
       ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
-          new String[] { Shell.WINUTILS, "help" });
+          new String[] { Shell.getWinutilsPath(), "help" });
       try {
         shellExecutor.execute();
       } catch (IOException e) {
@@ -75,9 +78,10 @@ public class WindowsBasedProcessTree extends ResourceCalculatorProcessTree {
 
   // helper method to override while testing
   String getAllProcessInfoFromShell() {
-    ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
-        new String[] { Shell.WINUTILS, "task", "processList", taskProcessId });
     try {
+      ShellCommandExecutor shellExecutor = new ShellCommandExecutor(
+          new String[] {Shell.getWinutilsFile().getCanonicalPath(),
+              "task", "processList", taskProcessId });
       shellExecutor.execute();
       return shellExecutor.getOutput();
     } catch (IOException e) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/ContainerExecutor.java

@@ -401,7 +401,7 @@ public abstract class ContainerExecutor implements Configurable {
           cpuRate = Math.min(10000, (int) (containerCpuPercentage * 100));
         }
       }
-      return new String[] { Shell.WINUTILS, "task", "create", "-m",
+      return new String[] { Shell.getWinutilsPath(), "task", "create", "-m",
           String.valueOf(memory), "-c", String.valueOf(cpuRate), groupId,
           "cmd /c " + command };
     } else {

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java

@@ -578,7 +578,8 @@ public class WindowsSecureContainerExecutor extends DefaultContainerExecutor {
       LOG.debug(String.format("getRunCommand: %s exists:%b", 
           command, f.exists()));
     }
-    return new String[] { Shell.WINUTILS, "task", "createAsUser", groupId, 
+    return new String[] { Shell.getWinutilsPath(), "task",
+        "createAsUser", groupId,
         userName, pidFile.toString(), "cmd /c " + command };
   }
   

+ 3 - 11
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

@@ -24,7 +24,6 @@ import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
 import java.io.DataOutputStream;
 import java.io.File;
 import java.io.IOException;
-import java.io.OutputStream;
 import java.io.PrintStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -747,16 +746,9 @@ public class ContainerLaunch implements Callable<Integer> {
       File srcFile = new File(src.toUri().getPath());
       String srcFileStr = srcFile.getPath();
       String dstFileStr = new File(dst.toString()).getPath();
-      // If not on Java7+ on Windows, then copy file instead of symlinking.
-      // See also FileUtil#symLink for full explanation.
-      if (!Shell.isJava7OrAbove() && srcFile.isFile()) {
-        lineWithLenCheck(String.format("@copy \"%s\" \"%s\"", srcFileStr, dstFileStr));
-        errorCheck();
-      } else {
-        lineWithLenCheck(String.format("@%s symlink \"%s\" \"%s\"", Shell.WINUTILS,
-          dstFileStr, srcFileStr));
-        errorCheck();
-      }
+      lineWithLenCheck(String.format("@%s symlink \"%s\" \"%s\"",
+          Shell.getWinutilsPath(), dstFileStr, srcFileStr));
+      errorCheck();
     }
 
     @Override

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerExecutor.java

@@ -34,6 +34,7 @@ import org.junit.Test;
 import static org.junit.Assert.*;
 import static org.junit.Assume.assumeTrue;
 
+@SuppressWarnings("deprecation")
 public class TestContainerExecutor {
   
   private ContainerExecutor containerExecutor = new DefaultContainerExecutor();

+ 15 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -978,7 +978,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Assume.assumeTrue(Shell.WINDOWS);
 
     // The tests are built on assuming 8191 max command line length
-    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGHT);
+    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGTH);
 
     ShellScriptBuilder builder = ShellScriptBuilder.create();
 
@@ -987,11 +987,11 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         org.apache.commons.lang.StringUtils.repeat("A", 1024)));
     builder.command(Arrays.asList(
         org.apache.commons.lang.StringUtils.repeat(
-            "E", Shell.WINDOWS_MAX_SHELL_LENGHT - callCmd.length())));
+            "E", Shell.WINDOWS_MAX_SHELL_LENGTH - callCmd.length())));
     try {
       builder.command(Arrays.asList(
           org.apache.commons.lang.StringUtils.repeat(
-              "X", Shell.WINDOWS_MAX_SHELL_LENGHT -callCmd.length() + 1)));
+              "X", Shell.WINDOWS_MAX_SHELL_LENGTH -callCmd.length() + 1)));
       fail("longCommand was expected to throw");
     } catch(IOException e) {
       assertThat(e.getMessage(), containsString(expectedMessage));
@@ -1026,17 +1026,17 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Assume.assumeTrue(Shell.WINDOWS);
 
     // The tests are built on assuming 8191 max command line length
-    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGHT);
+    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGTH);
 
     ShellScriptBuilder builder = ShellScriptBuilder.create();
 
     // test env
     builder.env("somekey", org.apache.commons.lang.StringUtils.repeat("A", 1024));
     builder.env("somekey", org.apache.commons.lang.StringUtils.repeat(
-        "A", Shell.WINDOWS_MAX_SHELL_LENGHT - ("@set somekey=").length()));
+        "A", Shell.WINDOWS_MAX_SHELL_LENGTH - ("@set somekey=").length()));
     try {
       builder.env("somekey", org.apache.commons.lang.StringUtils.repeat(
-          "A", Shell.WINDOWS_MAX_SHELL_LENGHT - ("@set somekey=").length()) + 1);
+          "A", Shell.WINDOWS_MAX_SHELL_LENGTH - ("@set somekey=").length()) + 1);
       fail("long env was expected to throw");
     } catch(IOException e) {
       assertThat(e.getMessage(), containsString(expectedMessage));
@@ -1051,17 +1051,17 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Assume.assumeTrue(Shell.WINDOWS);
 
     // The tests are built on assuming 8191 max command line length
-    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGHT);
+    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGTH);
 
     ShellScriptBuilder builder = ShellScriptBuilder.create();
 
     // test mkdir
     builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat("A", 1024)));
     builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat(
-        "E", (Shell.WINDOWS_MAX_SHELL_LENGHT - mkDirCmd.length())/2)));
+        "E", (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length())/2)));
     try {
       builder.mkdir(new Path(org.apache.commons.lang.StringUtils.repeat(
-          "X", (Shell.WINDOWS_MAX_SHELL_LENGHT - mkDirCmd.length())/2 +1)));
+          "X", (Shell.WINDOWS_MAX_SHELL_LENGTH - mkDirCmd.length())/2 +1)));
       fail("long mkdir was expected to throw");
     } catch(IOException e) {
       assertThat(e.getMessage(), containsString(expectedMessage));
@@ -1072,11 +1072,10 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
   public void testWindowsShellScriptBuilderLink() throws IOException {
     // Test is only relevant on Windows
     Assume.assumeTrue(Shell.WINDOWS);
-
-    String linkCmd = "@" +Shell.WINUTILS + " symlink \"\" \"\"";
+    String linkCmd = "@" + Shell.getWinutilsPath() + " symlink \"\" \"\"";
 
     // The tests are built on assuming 8191 max command line length
-    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGHT);
+    assertEquals(8191, Shell.WINDOWS_MAX_SHELL_LENGTH);
 
     ShellScriptBuilder builder = ShellScriptBuilder.create();
 
@@ -1085,15 +1084,15 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         new Path(org.apache.commons.lang.StringUtils.repeat("B", 1024)));
     builder.link(
         new Path(org.apache.commons.lang.StringUtils.repeat(
-            "E", (Shell.WINDOWS_MAX_SHELL_LENGHT - linkCmd.length())/2)),
+            "E", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2)),
         new Path(org.apache.commons.lang.StringUtils.repeat(
-            "F", (Shell.WINDOWS_MAX_SHELL_LENGHT - linkCmd.length())/2)));
+            "F", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2)));
     try {
       builder.link(
           new Path(org.apache.commons.lang.StringUtils.repeat(
-              "X", (Shell.WINDOWS_MAX_SHELL_LENGHT - linkCmd.length())/2 + 1)),
+              "X", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2 + 1)),
           new Path(org.apache.commons.lang.StringUtils.repeat(
-              "Y", (Shell.WINDOWS_MAX_SHELL_LENGHT - linkCmd.length())/2) + 1));
+              "Y", (Shell.WINDOWS_MAX_SHELL_LENGTH - linkCmd.length())/2) + 1));
       fail("long link was expected to throw");
     } catch(IOException e) {
       assertThat(e.getMessage(), containsString(expectedMessage));

部分文件因为文件数量过多而无法显示