Pārlūkot izejas kodu

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

Steve Loughran 10 gadi atpakaļ
vecāks
revīzija
5eab51a24f
19 mainītis faili ar 886 papildinājumiem un 365 dzēšanām
  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:");

Failā izmaiņas netiks attēlotas, jo tās ir par lielu
+ 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));

Daži faili netika attēloti, jo izmaiņu fails ir pārāk liels