Просмотр исходного кода

Merge r1476453 through r1477867 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-2802@1477868 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 12 лет назад
Родитель
Сommit
faca77f227
96 измененных файлов с 1784 добавлено и 743 удалено
  1. 28 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 126 4
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java
  3. 4 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java
  4. 23 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  5. 24 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java
  6. 18 9
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java
  7. 13 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java
  8. 37 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
  9. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsDynamicMBeanBase.java
  10. 12 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
  11. 5 57
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java
  12. 36 0
      hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c
  13. 11 0
      hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
  14. 5 0
      hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h
  15. 108 5
      hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c
  16. 6 6
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java
  17. 66 62
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java
  18. 135 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCopyPreserveFlag.java
  19. 13 2
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java
  20. 38 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java
  21. 2 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TemporarySocketDirectory.java
  22. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUserGroupInformation.java
  23. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java
  24. 42 10
      hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
  25. 3 3
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java
  26. 4 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java
  27. 17 2
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java
  28. 8 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
  29. 94 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSCustomUserName.java
  30. 15 10
      hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestUserProvider.java
  31. 42 4
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  32. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml
  33. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  34. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
  35. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
  36. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  37. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java
  38. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
  39. 18 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  40. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  41. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
  42. 17 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
  43. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  44. 16 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
  45. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java
  46. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java
  47. 16 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
  48. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
  49. 13 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
  50. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
  51. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java
  52. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java
  53. 14 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  54. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  55. 13 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
  56. 16 16
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java
  57. 58 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java
  58. 4 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java
  59. 50 21
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java
  60. 12 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java
  61. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java
  62. 17 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java
  63. 14 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java
  64. 34 12
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java
  65. 21 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java
  66. 11 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java
  67. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java
  68. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hdfs-site.xml
  69. 7 0
      hadoop-mapreduce-project/CHANGES.txt
  70. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java
  71. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java
  72. 1 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  73. 7 3
      hadoop-project/pom.xml
  74. 2 1
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PathFinder.java
  75. 2 1
      hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java
  76. 12 1
      hadoop-yarn-project/CHANGES.txt
  77. 4 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml
  78. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java
  79. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/util/CgroupsLCEResourcesHandler.java
  80. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
  81. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java
  82. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeHealthService.java
  83. 4 3
      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
  84. 44 37
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  85. 63 52
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  86. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java
  87. 0 43
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerSubmitEvent.java
  88. 21 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  89. 3 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  90. 110 133
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  91. 85 57
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  92. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  93. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
  94. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java
  95. 3 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml
  96. 1 4
      hadoop-yarn-project/hadoop-yarn/pom.xml

+ 28 - 1
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -382,7 +382,7 @@ Trunk (Unreleased)
 
     HADOOP-9190. packaging docs is broken. (Andy Isaacson via atm)
 
-  BREAKDOWN OF HADOOP-8562 SUBTASKS
+  BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
     HADOOP-8924. Hadoop Common creating package-info.java must not depend on
     sh. (Chris Nauroth via suresh)
@@ -519,6 +519,21 @@ Trunk (Unreleased)
 
     HADOOP-9443. Port winutils static code analysis change to trunk.
     (Chuan Liu via suresh)
+
+    HADOOP-9290. Some tests cannot load native library on windows.
+    (Chris Nauroth via suresh)
+
+    HADOOP-9500. TestUserGroupInformation#testGetServerSideGroups fails on 
+    Windows due to failure to find winutils.exe. (Chris Nauroth via suresh)
+
+    HADOOP-9490. LocalFileSystem#reportChecksumFailure not closing the 
+    checksum file handle before rename. (Ivan Mitic via suresh)
+
+    HADOOP-9524. Fix ShellCommandFencer to work on Windows.
+    (Arpit Agarwal via suresh)
+
+    HADOOP-9413. Add common utils for File#setReadable/Writable/Executable &
+    File#canRead/Write/Execute that work cross-platform. (Ivan Mitic via suresh)
     
 Release 2.0.5-beta - UNRELEASED
 
@@ -531,6 +546,9 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-8415. Add getDouble() and setDouble() in
     org.apache.hadoop.conf.Configuration (Jan van der Lugt via harsh)
 
+    HADOOP-9338. FsShell Copy Commands Should Optionally Preserve File
+    Attributes. (Nick White via atm)
+
   IMPROVEMENTS
 
     HADOOP-9253. Capture ulimit info in the logs at service start time.
@@ -562,6 +580,9 @@ Release 2.0.5-beta - UNRELEASED
     HADOOP-9503. Remove sleep between IPC client connect timeouts.
     (Varun Sharma via szetszwo)
 
+    HADOOP-9322. LdapGroupsMapping doesn't seem to set a timeout for
+    its directory search. (harsh)
+
   OPTIMIZATIONS
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs
@@ -644,6 +665,9 @@ Release 2.0.5-beta - UNRELEASED
 
     HADOOP-9473. Typo in FileUtil copy() method. (Glen Mazza via suresh)
 
+    HADOOP-9504. MetricsDynamicMBeanBase has concurrency issues in
+    createMBeanInfo (Liang Xie via jlowe)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -1657,6 +1681,9 @@ Release 0.23.8 - UNRELEASED
     HADOOP-9469. mapreduce/yarn source jars not included in dist tarball
     (Robert Parker via tgraves)
 
+    HADOOP-9504. MetricsDynamicMBeanBase has concurrency issues in
+    createMBeanInfo (Liang Xie via jlowe)
+
 Release 0.23.7 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 126 - 4
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -44,7 +44,6 @@ import org.apache.hadoop.io.nativeio.NativeIO;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
@@ -148,9 +147,9 @@ public class FileUtil {
    * Pure-Java implementation of "chmod +rwx f".
    */
   private static void grantPermissions(final File f) {
-      f.setExecutable(true);
-      f.setReadable(true);
-      f.setWritable(true);
+      FileUtil.setExecutable(f, true);
+      FileUtil.setReadable(f, true);
+      FileUtil.setWritable(f, true);
   }
 
   private static boolean deleteImpl(final File f, final boolean doLog) {
@@ -851,6 +850,129 @@ public class FileUtil {
     execCommand(file, cmd);
   }
 
+  /**
+   * Platform independent implementation for {@link File#setReadable(boolean)}
+   * File#setReadable does not work as expected on Windows.
+   * @param f input file
+   * @param readable
+   * @return true on success, false otherwise
+   */
+  public static boolean setReadable(File f, boolean readable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = readable ? "u+r" : "u-r";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setReadable(readable);
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#setWritable(boolean)}
+   * File#setWritable does not work as expected on Windows.
+   * @param f input file
+   * @param writable
+   * @return true on success, false otherwise
+   */
+  public static boolean setWritable(File f, boolean writable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = writable ? "u+w" : "u-w";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setWritable(writable);
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#setExecutable(boolean)}
+   * File#setExecutable does not work as expected on Windows.
+   * Note: revoking execute permission on folders does not have the same
+   * behavior on Windows as on Unix platforms. Creating, deleting or renaming
+   * a file within that folder will still succeed on Windows.
+   * @param f input file
+   * @param executable
+   * @return true on success, false otherwise
+   */
+  public static boolean setExecutable(File f, boolean executable) {
+    if (Shell.WINDOWS) {
+      try {
+        String permission = executable ? "u+x" : "u-x";
+        FileUtil.chmod(f.getCanonicalPath(), permission, false);
+        return true;
+      } catch (IOException ex) {
+        return false;
+      }
+    } else {
+      return f.setExecutable(executable);
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canRead()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canRead()}
+   *         On Windows, true if process has read access on the path
+   */
+  public static boolean canRead(File f) {
+    if (Shell.WINDOWS) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_READ);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canRead();
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canWrite()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canWrite()}
+   *         On Windows, true if process has write access on the path
+   */
+  public static boolean canWrite(File f) {
+    if (Shell.WINDOWS) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_WRITE);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canWrite();
+    }
+  }
+
+  /**
+   * Platform independent implementation for {@link File#canExecute()}
+   * @param f input file
+   * @return On Unix, same as {@link File#canExecute()}
+   *         On Windows, true if process has execute access on the path
+   */
+  public static boolean canExecute(File f) {
+    if (Shell.WINDOWS) {
+      try {
+        return NativeIO.Windows.access(f.getCanonicalPath(),
+            NativeIO.Windows.AccessRight.ACCESS_EXECUTE);
+      } catch (IOException e) {
+        return false;
+      }
+    } else {
+      return f.canExecute();
+    }
+  }
+
   /**
    * Set permissions to the required value. Uses the java primitives instead
    * of forking if group == other.

+ 4 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java

@@ -103,7 +103,8 @@ public class LocalFileSystem extends ChecksumFileSystem {
       String device = new DF(f, getConf()).getMount();
       File parent = f.getParentFile();
       File dir = null;
-      while (parent!=null && parent.canWrite() && parent.toString().startsWith(device)) {
+      while (parent != null && FileUtil.canWrite(parent) &&
+          parent.toString().startsWith(device)) {
         dir = parent;
         parent = parent.getParentFile();
       }
@@ -130,6 +131,8 @@ public class LocalFileSystem extends ChecksumFileSystem {
       }
       // move checksum file too
       File checkFile = ((RawLocalFileSystem)fs).pathToFile(getChecksumFile(p));
+      // close the stream before rename to release the file handle
+      sums.close();
       b = checkFile.renameTo(new File(badDir, checkFile.getName()+suffix));
       if (!b) {
           LOG.warn("Ignoring failure of renameTo");

+ 23 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -618,4 +618,27 @@ public class RawLocalFileSystem extends FileSystem {
         FileUtil.makeShellPath(pathToFile(p), true)));
     }
   }
+ 
+  /**
+   * Sets the {@link Path}'s last modified time <em>only</em> to the given
+   * valid time.
+   *
+   * @param mtime the modification time to set (only if greater than zero).
+   * @param atime currently ignored.
+   * @throws IOException if setting the last modified time fails.
+   */
+  @Override
+  public void setTimes(Path p, long mtime, long atime) throws IOException {
+    File f = pathToFile(p);
+    if(mtime >= 0) {
+      if(!f.setLastModified(mtime)) {
+        throw new IOException(
+          "couldn't set last-modified time to " +
+          mtime +
+          " for " +
+          f.getAbsolutePath());
+      }
+    }
+  }
+
 }

+ 24 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.io.IOUtils;
 abstract class CommandWithDestination extends FsCommand {  
   protected PathData dst;
   private boolean overwrite = false;
+  private boolean preserve = false;
   private boolean verifyChecksum = true;
   private boolean writeChecksum = true;
   
@@ -66,6 +67,16 @@ abstract class CommandWithDestination extends FsCommand {
     writeChecksum = flag;
   }
   
+  /**
+   * If true, the last modified time, last access time,
+   * owner, group and permission information of the source
+   * file will be preserved as far as target {@link FileSystem}
+   * implementation allows.
+   */
+  protected void setPreserve(boolean preserve) {
+    this.preserve = preserve;
+  }
+
   /**
    *  The last arg is expected to be a local path, if only one argument is
    *  given then the destination will be the current directory 
@@ -227,6 +238,19 @@ abstract class CommandWithDestination extends FsCommand {
     try {
       in = src.fs.open(src.path);
       copyStreamToTarget(in, target);
+      if(preserve) {
+        target.fs.setTimes(
+          target.path,
+          src.stat.getModificationTime(),
+          src.stat.getAccessTime());
+        target.fs.setOwner(
+          target.path,
+          src.stat.getOwner(),
+          src.stat.getGroup());
+        target.fs.setPermission(
+          target.path,
+          src.stat.getPermission());
+      }
     } finally {
       IOUtils.closeStream(in);
     }

+ 18 - 9
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CopyCommands.java

@@ -129,17 +129,19 @@ class CopyCommands {
 
   static class Cp extends CommandWithDestination {
     public static final String NAME = "cp";
-    public static final String USAGE = "<src> ... <dst>";
+    public static final String USAGE = "[-f] [-p] <src> ... <dst>";
     public static final String DESCRIPTION =
       "Copy files that match the file pattern <src> to a\n" +
       "destination.  When copying multiple files, the destination\n" +
-      "must be a directory.";
+      "must be a directory. Passing -p preserves access and\n" +
+      "modification times, ownership and the mode.\n";
     
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "f");
+      CommandFormat cf = new CommandFormat(2, Integer.MAX_VALUE, "f", "p");
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
+      setPreserve(cf.getOpt("p"));
       // should have a -r option
       setRecursive(true);
       getRemoteDestination(args);
@@ -152,20 +154,23 @@ class CopyCommands {
   public static class Get extends CommandWithDestination {
     public static final String NAME = "get";
     public static final String USAGE =
-      "[-ignoreCrc] [-crc] <src> ... <localdst>";
+      "[-p] [-ignoreCrc] [-crc] <src> ... <localdst>";
     public static final String DESCRIPTION =
       "Copy files that match the file pattern <src>\n" +
       "to the local name.  <src> is kept.  When copying multiple,\n" +
-      "files, the destination must be a directory.";
+      "files, the destination must be a directory. Passing\n" +
+      "-p preserves access and modification times,\n" +
+      "ownership and the mode.\n";
 
     @Override
     protected void processOptions(LinkedList<String> args)
     throws IOException {
       CommandFormat cf = new CommandFormat(
-          1, Integer.MAX_VALUE, "crc", "ignoreCrc");
+          1, Integer.MAX_VALUE, "crc", "ignoreCrc", "p");
       cf.parse(args);
       setWriteChecksum(cf.getOpt("crc"));
       setVerifyChecksum(!cf.getOpt("ignoreCrc"));
+      setPreserve(cf.getOpt("p"));
       setRecursive(true);
       getLocalDestination(args);
     }
@@ -176,16 +181,20 @@ class CopyCommands {
    */
   public static class Put extends CommandWithDestination {
     public static final String NAME = "put";
-    public static final String USAGE = "<localsrc> ... <dst>";
+    public static final String USAGE = "[-f] [-p] <localsrc> ... <dst>";
     public static final String DESCRIPTION =
       "Copy files from the local file system\n" +
-      "into fs.";
+      "into fs. Copying fails if the file already\n" +
+      "exists, unless the -f flag is given. Passing\n" +
+      "-p preserves access and modification times,\n" +
+      "ownership and the mode.\n";
 
     @Override
     protected void processOptions(LinkedList<String> args) throws IOException {
-      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE, "f");
+      CommandFormat cf = new CommandFormat(1, Integer.MAX_VALUE, "f", "p");
       cf.parse(args);
       setOverwrite(cf.getOpt("f"));
+      setPreserve(cf.getOpt("p"));
       getRemoteDestination(args);
       // should have a -r option
       setRecursive(true);

+ 13 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ShellCommandFencer.java

@@ -26,6 +26,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configured;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.util.Shell;
 
 /**
  * Fencing method that runs a shell command. It should be specified
@@ -33,8 +34,8 @@ import com.google.common.annotations.VisibleForTesting;
  * <code>
  *   shell(/path/to/my/script.sh arg1 arg2 ...)
  * </code><br>
- * The string between '(' and ')' is passed directly to a bash shell and
- * may not include any closing parentheses.<p>
+ * The string between '(' and ')' is passed directly to a bash shell
+ * (cmd.exe on Windows) and may not include any closing parentheses.<p>
  * 
  * The shell command will be run with an environment set up to contain
  * all of the current Hadoop configuration variables, with the '_' character 
@@ -58,11 +59,11 @@ public class ShellCommandFencer
 
   /** Prefix for target parameters added to the environment */
   private static final String TARGET_PREFIX = "target_";
-  
+
   @VisibleForTesting
   static Log LOG = LogFactory.getLog(
       ShellCommandFencer.class);
-  
+
   @Override
   public void checkArgs(String args) throws BadFencingConfigurationException {
     if (args == null || args.isEmpty()) {
@@ -74,8 +75,14 @@ public class ShellCommandFencer
 
   @Override
   public boolean tryFence(HAServiceTarget target, String cmd) {
-    ProcessBuilder builder = new ProcessBuilder(
-        "bash", "-e", "-c", cmd);
+    ProcessBuilder builder;
+
+    if (!Shell.WINDOWS) {
+      builder = new ProcessBuilder("bash", "-e", "-c", cmd);
+    } else {
+      builder = new ProcessBuilder("cmd.exe", "/c", cmd);
+    }
+
     setConfAsEnvVars(builder.environment());
     addTargetInfoAsEnvVars(target, builder.environment());
 

+ 37 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java

@@ -356,6 +356,43 @@ public class NativeIO {
     /** Windows only methods used for getOwner() implementation */
     private static native String getOwner(FileDescriptor fd) throws IOException;
 
+    /** Supported list of Windows access right flags */
+    public static enum AccessRight {
+      ACCESS_READ (0x0001),      // FILE_READ_DATA
+      ACCESS_WRITE (0x0002),     // FILE_WRITE_DATA
+      ACCESS_EXECUTE (0x0020);   // FILE_EXECUTE
+
+      private final int accessRight;
+      AccessRight(int access) {
+        accessRight = access;
+      }
+
+      public int accessRight() {
+        return accessRight;
+      }
+    };
+
+    /** Windows only method used to check if the current process has requested
+     *  access rights on the given path. */
+    private static native boolean access0(String path, int requestedAccess);
+
+    /**
+     * Checks whether the current process has desired access rights on
+     * the given path.
+     * 
+     * Longer term this native function can be substituted with JDK7
+     * function Files#isReadable, isWritable, isExecutable.
+     *
+     * @param path input path
+     * @param desiredAccess ACCESS_READ, ACCESS_WRITE or ACCESS_EXECUTE
+     * @return true if access is allowed
+     * @throws IOException I/O exception on error
+     */
+    public static boolean access(String path, AccessRight desiredAccess)
+        throws IOException {
+      return access0(path, desiredAccess.accessRight());
+    }
+
     static {
       if (NativeCodeLoader.isNativeCodeLoaded()) {
         try {

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/util/MetricsDynamicMBeanBase.java

@@ -18,9 +18,9 @@
 package org.apache.hadoop.metrics.util;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
 
 import javax.management.Attribute;
 import javax.management.AttributeList;
@@ -69,6 +69,7 @@ public abstract class MetricsDynamicMBeanBase implements DynamicMBean {
   protected MetricsDynamicMBeanBase(final MetricsRegistry mr, final String aMBeanDescription) {
     metricsRegistry = mr;
     mbeanDescription = aMBeanDescription;
+    metricsRateAttributeMod = new ConcurrentHashMap<String, MetricsBase>();
     createMBeanInfo();
   }
   
@@ -78,7 +79,6 @@ public abstract class MetricsDynamicMBeanBase implements DynamicMBean {
   }
   
   private void createMBeanInfo() {
-    metricsRateAttributeMod = new HashMap<String, MetricsBase>();
     boolean needsMinMaxResetOperation = false;
     List<MBeanAttributeInfo> attributesInfo = new ArrayList<MBeanAttributeInfo>();
     MBeanOperationInfo[] operationsInfo = null;

+ 12 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java

@@ -144,7 +144,15 @@ public class LdapGroupsMapping
    */
   public static final String GROUP_NAME_ATTR_KEY = LDAP_CONFIG_PREFIX + ".search.attr.group.name";
   public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
-  
+
+  /*
+   * LDAP {@link SearchControls} attribute to set the time limit
+   * for an invoked directory search. Prevents infinite wait cases.
+   */
+  public static final String DIRECTORY_SEARCH_TIMEOUT =
+    LDAP_CONFIG_PREFIX + ".directory.search.timeout";
+  public static final int DIRECTORY_SEARCH_TIMEOUT_DEFAULT = 10000; // 10s
+
   private static final Log LOG = LogFactory.getLog(LdapGroupsMapping.class);
 
   private static final SearchControls SEARCH_CONTROLS = new SearchControls();
@@ -326,6 +334,9 @@ public class LdapGroupsMapping
     groupNameAttr =
         conf.get(GROUP_NAME_ATTR_KEY, GROUP_NAME_ATTR_DEFAULT);
 
+    int dirSearchTimeout = conf.getInt(DIRECTORY_SEARCH_TIMEOUT, DIRECTORY_SEARCH_TIMEOUT_DEFAULT);
+    SEARCH_CONTROLS.setTimeLimit(dirSearchTimeout);
+
     this.conf = conf;
   }
   

+ 5 - 57
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
@@ -160,11 +161,7 @@ public class DiskChecker {
                                    + dir.toString());
     }
 
-    if (Shell.WINDOWS) {
-      checkAccessByFileSystemInteraction(dir);
-    } else {
-      checkAccessByFileMethods(dir);
-    }
+    checkAccessByFileMethods(dir);
   }
 
   /**
@@ -177,68 +174,19 @@ public class DiskChecker {
    */
   private static void checkAccessByFileMethods(File dir)
       throws DiskErrorException {
-    if (!dir.canRead()) {
+    if (!FileUtil.canRead(dir)) {
       throw new DiskErrorException("Directory is not readable: "
                                    + dir.toString());
     }
 
-    if (!dir.canWrite()) {
+    if (!FileUtil.canWrite(dir)) {
       throw new DiskErrorException("Directory is not writable: "
                                    + dir.toString());
     }
 
-    if (!dir.canExecute()) {
+    if (!FileUtil.canExecute(dir)) {
       throw new DiskErrorException("Directory is not executable: "
                                    + dir.toString());
     }
   }
-
-  /**
-   * Checks that the current running process can read, write, and execute the
-   * given directory by attempting each of those operations on the file system.
-   * This method contains several workarounds to known JVM bugs that cause
-   * File.canRead, File.canWrite, and File.canExecute to return incorrect results
-   * on Windows with NTFS ACLs.  See:
-   * http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6203387
-   * These bugs are supposed to be fixed in JDK7.
-   * 
-   * @param dir File to check
-   * @throws DiskErrorException if dir is not readable, not writable, or not
-   *   executable
-   */
-  private static void checkAccessByFileSystemInteraction(File dir)
-      throws DiskErrorException {
-    // Make sure we can read the directory by listing it.
-    if (dir.list() == null) {
-      throw new DiskErrorException("Directory is not readable: "
-                                   + dir.toString());
-    }
-
-    // Make sure we can write to the directory by creating a temp file in it.
-    try {
-      File tempFile = File.createTempFile("checkDirAccess", null, dir);
-      if (!tempFile.delete()) {
-        throw new DiskErrorException("Directory is not writable: "
-                                     + dir.toString());
-      }
-    } catch (IOException e) {
-      throw new DiskErrorException("Directory is not writable: "
-                                   + dir.toString(), e);
-    }
-
-    // Make sure the directory is executable by trying to cd into it.  This
-    // launches a separate process.  It does not change the working directory of
-    // the current process.
-    try {
-      String[] cdCmd = new String[] { "cmd", "/C", "cd",
-          dir.getAbsolutePath() };
-      Shell.execCommand(null, cdCmd, SHELL_TIMEOUT);
-    } catch (Shell.ExitCodeException e) {
-      throw new DiskErrorException("Directory is not executable: "
-                                   + dir.toString(), e);
-    } catch (IOException e) {
-      throw new DiskErrorException("Directory is not executable: "
-                                   + dir.toString(), e);
-    }
-  }
 }

+ 36 - 0
hadoop-common-project/hadoop-common/src/main/native/src/org/apache/hadoop/io/nativeio/NativeIO.c

@@ -812,6 +812,42 @@ cleanup:
 #endif
 }
 
+/*
+ * Class:     org_apache_hadoop_io_nativeio_NativeIO_Windows
+ * Method:    access0
+ * Signature: (Ljava/lang/String;I)Z
+ */
+JNIEXPORT jboolean JNICALL Java_org_apache_hadoop_io_nativeio_NativeIO_00024Windows_access0
+  (JNIEnv *env, jclass clazz, jstring jpath, jint jaccess)
+{
+#ifdef UNIX
+  THROW(env, "java/io/IOException",
+    "The function access0(path, access) is not supported on Unix");
+  return NULL;
+#endif
+
+#ifdef WINDOWS
+  LPCWSTR path = NULL;
+  DWORD dwRtnCode = ERROR_SUCCESS;
+  ACCESS_MASK access = (ACCESS_MASK)jaccess;
+  BOOL allowed = FALSE;
+
+  path = (LPCWSTR) (*env)->GetStringChars(env, jpath, NULL);
+  if (!path) goto cleanup; // exception was thrown
+
+  dwRtnCode = CheckAccessForCurrentUser(path, access, &allowed);
+  if (dwRtnCode != ERROR_SUCCESS) {
+    throw_ioe(env, dwRtnCode);
+    goto cleanup;
+  }
+
+cleanup:
+  if (path) (*env)->ReleaseStringChars(env, jpath, path);
+
+  return (jboolean)allowed;
+#endif
+}
+
 JNIEXPORT void JNICALL 
 Java_org_apache_hadoop_io_nativeio_NativeIO_renameTo0(JNIEnv *env, 
 jclass clazz, jstring jsrc, jstring jdst)

+ 11 - 0
hadoop-common-project/hadoop-common/src/main/resources/core-default.xml

@@ -212,6 +212,17 @@
   </description>
 </property>
 
+<property>
+  <name>hadoop.security.group.mapping.ldap.directory.search.timeout</name>
+  <value>10000</value>
+  <description>
+    The attribute applied to the LDAP SearchControl properties to set a
+    maximum time limit when searching and awaiting a result.
+    Set to 0 if infinite wait period is desired.
+    Default is 10 seconds. Units in milliseconds.
+  </description>
+</property>
+
 <property>
   <name>hadoop.security.service.user.name.key</name>
   <value></value>

+ 5 - 0
hadoop-common-project/hadoop-common/src/main/winutils/include/winutils.h

@@ -110,6 +110,11 @@ void SystemInfoUsage();
 DWORD GetFileInformationByName(__in LPCWSTR pathName,  __in BOOL followLink,
   __out LPBY_HANDLE_FILE_INFORMATION lpFileInformation);
 
+DWORD CheckAccessForCurrentUser(
+  __in PCWSTR pathName,
+  __in ACCESS_MASK requestedAccess,
+  __out BOOL *allowed);
+
 DWORD ConvertToLongPath(__in PCWSTR path, __deref_out PWSTR *newPath);
 
 DWORD GetSidFromAcctNameW(__in PCWSTR acctName, __out PSID* ppSid);

+ 108 - 5
hadoop-common-project/hadoop-common/src/main/winutils/libwinutils.c

@@ -567,7 +567,7 @@ static DWORD GetEffectiveRightsForSid(PSECURITY_DESCRIPTOR psd,
   PSID pSid,
   PACCESS_MASK pAccessRights)
 {
-  AUTHZ_RESOURCE_MANAGER_HANDLE hManager;
+  AUTHZ_RESOURCE_MANAGER_HANDLE hManager = NULL;
   LUID unusedId = { 0 };
   AUTHZ_CLIENT_CONTEXT_HANDLE hAuthzClientContext = NULL;
   DWORD dwRtnCode = ERROR_SUCCESS;
@@ -581,6 +581,10 @@ static DWORD GetEffectiveRightsForSid(PSECURITY_DESCRIPTOR psd,
     return GetLastError();
   }
 
+  // Pass AUTHZ_SKIP_TOKEN_GROUPS to the function to avoid querying user group
+  // information for access check. This allows us to model POSIX permissions
+  // on Windows, where a user can have less permissions than a group it
+  // belongs to.
   if(!AuthzInitializeContextFromSid(AUTHZ_SKIP_TOKEN_GROUPS,
     pSid, hManager, NULL, unusedId, NULL, &hAuthzClientContext))
   {
@@ -594,16 +598,115 @@ static DWORD GetEffectiveRightsForSid(PSECURITY_DESCRIPTOR psd,
     ret = dwRtnCode;
     goto GetEffectiveRightsForSidEnd;
   }
-  if (!AuthzFreeContext(hAuthzClientContext))
+
+GetEffectiveRightsForSidEnd:
+  if (hManager != NULL)
   {
-    ret = GetLastError();
-    goto GetEffectiveRightsForSidEnd;
+    (void)AuthzFreeResourceManager(hManager);
+  }
+  if (hAuthzClientContext != NULL)
+  {
+    (void)AuthzFreeContext(hAuthzClientContext);
   }
 
-GetEffectiveRightsForSidEnd:
   return ret;
 }
 
+//----------------------------------------------------------------------------
+// Function: CheckAccessForCurrentUser
+//
+// Description:
+//   Checks if the current process has the requested access rights on the given
+//   path. Based on the following MSDN article:
+//   http://msdn.microsoft.com/en-us/library/windows/desktop/ff394771(v=vs.85).aspx
+//
+// Returns:
+//   ERROR_SUCCESS: on success
+//
+DWORD CheckAccessForCurrentUser(
+  __in PCWSTR pathName,
+  __in ACCESS_MASK requestedAccess,
+  __out BOOL *allowed)
+{
+  DWORD dwRtnCode = ERROR_SUCCESS;
+
+  LPWSTR longPathName = NULL;
+  HANDLE hProcessToken = NULL;
+  PSECURITY_DESCRIPTOR pSd = NULL;
+
+  AUTHZ_RESOURCE_MANAGER_HANDLE hManager = NULL;
+  AUTHZ_CLIENT_CONTEXT_HANDLE hAuthzClientContext = NULL;
+  LUID Luid = {0, 0};
+
+  ACCESS_MASK currentUserAccessRights = 0;
+
+  // Prepend the long path prefix if needed
+  dwRtnCode = ConvertToLongPath(pathName, &longPathName);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  // Get SD of the given path. OWNER and DACL security info must be
+  // requested, otherwise, AuthzAccessCheck fails with invalid parameter
+  // error.
+  dwRtnCode = GetNamedSecurityInfo(longPathName, SE_FILE_OBJECT,
+    OWNER_SECURITY_INFORMATION | GROUP_SECURITY_INFORMATION |
+    DACL_SECURITY_INFORMATION,
+    NULL, NULL, NULL, NULL, &pSd);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  // Get current process token
+  if (!OpenProcessToken(GetCurrentProcess(), TOKEN_QUERY, &hProcessToken))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  if (!AuthzInitializeResourceManager(AUTHZ_RM_FLAG_NO_AUDIT, NULL, NULL,
+    NULL, NULL, &hManager))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  if(!AuthzInitializeContextFromToken(0, hProcessToken, hManager, NULL,
+    Luid, NULL, &hAuthzClientContext))
+  {
+    dwRtnCode = GetLastError();
+    goto CheckAccessEnd;
+  }
+
+  dwRtnCode = GetAccess(hAuthzClientContext, pSd, &currentUserAccessRights);
+  if (dwRtnCode != ERROR_SUCCESS)
+  {
+    goto CheckAccessEnd;
+  }
+
+  *allowed = ((currentUserAccessRights & requestedAccess) == requestedAccess);
+
+CheckAccessEnd:
+  LocalFree(longPathName);
+  LocalFree(pSd);
+  if (hProcessToken != NULL)
+  {
+    CloseHandle(hProcessToken);
+  }
+  if (hManager != NULL)
+  {
+    (void)AuthzFreeResourceManager(hManager);
+  }
+  if (hAuthzClientContext != NULL)
+  {
+    (void)AuthzFreeContext(hAuthzClientContext);
+  }
+
+  return dwRtnCode;
+}
+
 //----------------------------------------------------------------------------
 // Function: FindFileOwnerAndPermission
 //

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

@@ -353,15 +353,15 @@ public class TestFileUtil {
   }
   
   private static void grantPermissions(final File f) {
-    f.setReadable(true);
-    f.setWritable(true);
-    f.setExecutable(true);
+    FileUtil.setReadable(f, true);
+    FileUtil.setWritable(f, true);
+    FileUtil.setExecutable(f, true);
   }
   
   private static void revokePermissions(final File f) {
-     f.setWritable(false);
-     f.setExecutable(false);
-     f.setReadable(false);
+     FileUtil.setWritable(f, false);
+     FileUtil.setExecutable(f, false);
+     FileUtil.setReadable(f, false);
   }
   
   // Validates the return value.

+ 66 - 62
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java

@@ -52,14 +52,15 @@ public class TestLocalFileSystem {
   
   @Before
   public void setup() throws IOException {
-    conf = new Configuration();
+    conf = new Configuration(false);
+    conf.set("fs.file.impl", LocalFileSystem.class.getName());
     fileSys = FileSystem.getLocal(conf);
     fileSys.delete(new Path(TEST_ROOT_DIR), true);
   }
   
   @After
   public void after() throws IOException {
-    base.setWritable(true);
+    FileUtil.setWritable(base, true);
     FileUtil.fullyDelete(base);
     assertTrue(!base.exists());
   }
@@ -67,7 +68,7 @@ public class TestLocalFileSystem {
   /**
    * Test the capability of setting the working directory.
    */
-  @Test
+  @Test(timeout = 1000)
   public void testWorkingDirectory() throws IOException {
     Path origDir = fileSys.getWorkingDirectory();
     Path subdir = new Path(TEST_ROOT_DIR, "new");
@@ -121,10 +122,9 @@ public class TestLocalFileSystem {
    * test Syncable interface on raw local file system
    * @throws IOException
    */
-  @Test
+  @Test(timeout = 1000)
   public void testSyncable() throws IOException {
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf).getRawFileSystem();
+    FileSystem fs = fileSys.getRawFileSystem();
     Path file = new Path(TEST_ROOT_DIR, "syncable");
     FSDataOutputStream out = fs.create(file);;
     final int bytesWritten = 1;
@@ -155,76 +155,68 @@ public class TestLocalFileSystem {
     }
   }
   
-  @Test
+  @Test(timeout = 1000)
   public void testCopy() throws IOException {
-    Configuration conf = new Configuration();
-    LocalFileSystem fs = FileSystem.getLocal(conf);
     Path src = new Path(TEST_ROOT_DIR, "dingo");
     Path dst = new Path(TEST_ROOT_DIR, "yak");
-    writeFile(fs, src, 1);
-    assertTrue(FileUtil.copy(fs, src, fs, dst, true, false, conf));
-    assertTrue(!fs.exists(src) && fs.exists(dst));
-    assertTrue(FileUtil.copy(fs, dst, fs, src, false, false, conf));
-    assertTrue(fs.exists(src) && fs.exists(dst));
-    assertTrue(FileUtil.copy(fs, src, fs, dst, true, true, conf));
-    assertTrue(!fs.exists(src) && fs.exists(dst));
-    fs.mkdirs(src);
-    assertTrue(FileUtil.copy(fs, dst, fs, src, false, false, conf));
+    writeFile(fileSys, src, 1);
+    assertTrue(FileUtil.copy(fileSys, src, fileSys, dst, true, false, conf));
+    assertTrue(!fileSys.exists(src) && fileSys.exists(dst));
+    assertTrue(FileUtil.copy(fileSys, dst, fileSys, src, false, false, conf));
+    assertTrue(fileSys.exists(src) && fileSys.exists(dst));
+    assertTrue(FileUtil.copy(fileSys, src, fileSys, dst, true, true, conf));
+    assertTrue(!fileSys.exists(src) && fileSys.exists(dst));
+    fileSys.mkdirs(src);
+    assertTrue(FileUtil.copy(fileSys, dst, fileSys, src, false, false, conf));
     Path tmp = new Path(src, dst.getName());
-    assertTrue(fs.exists(tmp) && fs.exists(dst));
-    assertTrue(FileUtil.copy(fs, dst, fs, src, false, true, conf));
-    assertTrue(fs.delete(tmp, true));
-    fs.mkdirs(tmp);
+    assertTrue(fileSys.exists(tmp) && fileSys.exists(dst));
+    assertTrue(FileUtil.copy(fileSys, dst, fileSys, src, false, true, conf));
+    assertTrue(fileSys.delete(tmp, true));
+    fileSys.mkdirs(tmp);
     try {
-      FileUtil.copy(fs, dst, fs, src, true, true, conf);
+      FileUtil.copy(fileSys, dst, fileSys, src, true, true, conf);
       fail("Failed to detect existing dir");
     } catch (IOException e) {
       // Expected
     }
   }
 
-  @Test
+  @Test(timeout = 1000)
   public void testHomeDirectory() throws IOException {
-    Configuration conf = new Configuration();
-    FileSystem fileSys = FileSystem.getLocal(conf);
     Path home = new Path(System.getProperty("user.home"))
       .makeQualified(fileSys);
     Path fsHome = fileSys.getHomeDirectory();
     assertEquals(home, fsHome);
   }
 
-  @Test
+  @Test(timeout = 1000)
   public void testPathEscapes() throws IOException {
-    Configuration conf = new Configuration();
-    FileSystem fs = FileSystem.getLocal(conf);
     Path path = new Path(TEST_ROOT_DIR, "foo%bar");
-    writeFile(fs, path, 1);
-    FileStatus status = fs.getFileStatus(path);
-    assertEquals(path.makeQualified(fs), status.getPath());
-    cleanupFile(fs, path);
+    writeFile(fileSys, path, 1);
+    FileStatus status = fileSys.getFileStatus(path);
+    assertEquals(path.makeQualified(fileSys), status.getPath());
+    cleanupFile(fileSys, path);
   }
   
-  @Test
+  @Test(timeout = 1000)
   public void testMkdirs() throws IOException {
-    Configuration conf = new Configuration();
-    LocalFileSystem fs = FileSystem.getLocal(conf);
     Path test_dir = new Path(TEST_ROOT_DIR, "test_dir");
     Path test_file = new Path(TEST_ROOT_DIR, "file1");
-    assertTrue(fs.mkdirs(test_dir));
+    assertTrue(fileSys.mkdirs(test_dir));
    
-    writeFile(fs, test_file, 1);
+    writeFile(fileSys, test_file, 1);
     // creating dir over a file
     Path bad_dir = new Path(test_file, "another_dir");
     
     try {
-      fs.mkdirs(bad_dir);
+      fileSys.mkdirs(bad_dir);
       fail("Failed to detect existing file in path");
     } catch (FileAlreadyExistsException e) { 
       // Expected
     }
     
     try {
-      fs.mkdirs(null);
+        fileSys.mkdirs(null);
       fail("Failed to detect null in mkdir arg");
     } catch (IllegalArgumentException e) {
       // Expected
@@ -232,26 +224,23 @@ public class TestLocalFileSystem {
   }
 
   /** Test deleting a file, directory, and non-existent path */
-  @Test
+  @Test(timeout = 1000)
   public void testBasicDelete() throws IOException {
-    Configuration conf = new Configuration();
-    LocalFileSystem fs = FileSystem.getLocal(conf);
     Path dir1 = new Path(TEST_ROOT_DIR, "dir1");
     Path file1 = new Path(TEST_ROOT_DIR, "file1");
     Path file2 = new Path(TEST_ROOT_DIR+"/dir1", "file2");
     Path file3 = new Path(TEST_ROOT_DIR, "does-not-exist");
-    assertTrue(fs.mkdirs(dir1));
-    writeFile(fs, file1, 1);
-    writeFile(fs, file2, 1);
+    assertTrue(fileSys.mkdirs(dir1));
+    writeFile(fileSys, file1, 1);
+    writeFile(fileSys, file2, 1);
     assertFalse("Returned true deleting non-existant path", 
-        fs.delete(file3));
-    assertTrue("Did not delete file", fs.delete(file1));
-    assertTrue("Did not delete non-empty dir", fs.delete(dir1));
+            fileSys.delete(file3));
+    assertTrue("Did not delete file", fileSys.delete(file1));
+    assertTrue("Did not delete non-empty dir", fileSys.delete(dir1));
   }
   
-  @Test
+  @Test(timeout = 1000)
   public void testStatistics() throws Exception {
-    FileSystem.getLocal(new Configuration());
     int fileSchemeCount = 0;
     for (Statistics stats : FileSystem.getAllStatistics()) {
       if (stats.getScheme().equals("file")) {
@@ -261,12 +250,10 @@ public class TestLocalFileSystem {
     assertEquals(1, fileSchemeCount);
   }
 
-  @Test
+  @Test(timeout = 1000)
   public void testHasFileDescriptor() throws IOException {
-    Configuration conf = new Configuration();
-    LocalFileSystem fs = FileSystem.getLocal(conf);
     Path path = new Path(TEST_ROOT_DIR, "test-file");
-    writeFile(fs, path, 1);
+    writeFile(fileSys, path, 1);
     BufferedFSInputStream bis = null;
     try {
       bis = new BufferedFSInputStream(new RawLocalFileSystem()
@@ -277,20 +264,18 @@ public class TestLocalFileSystem {
     }
   }
 
-  @Test
+  @Test(timeout = 1000)
   public void testListStatusWithColons() throws IOException {
     assumeTrue(!Shell.WINDOWS);
-    Configuration conf = new Configuration();
-    LocalFileSystem fs = FileSystem.getLocal(conf);
     File colonFile = new File(TEST_ROOT_DIR, "foo:bar");
     colonFile.mkdirs();
-    FileStatus[] stats = fs.listStatus(new Path(TEST_ROOT_DIR));
+    FileStatus[] stats = fileSys.listStatus(new Path(TEST_ROOT_DIR));
     assertEquals("Unexpected number of stats", 1, stats.length);
     assertEquals("Bad path from stat", colonFile.getAbsolutePath(),
         stats[0].getPath().toUri().getPath());
   }
   
-  @Test
+  @Test(timeout = 1000)
   public void testReportChecksumFailure() throws IOException {
     base.mkdirs();
     assertTrue(base.exists() && base.isDirectory());
@@ -298,7 +283,7 @@ public class TestLocalFileSystem {
     final File dir1 = new File(base, "dir1");
     final File dir2 = new File(dir1, "dir2");
     dir2.mkdirs();
-    assertTrue(dir2.exists() && dir2.canWrite());
+    assertTrue(dir2.exists() && FileUtil.canWrite(dir2));
     
     final String dataFileName = "corruptedData";
     final Path dataPath = new Path(new File(dir2, dataFileName).toURI());
@@ -321,7 +306,7 @@ public class TestLocalFileSystem {
     // this is a hack to force the #reportChecksumFailure() method to stop
     // climbing up at the 'base' directory and use 'dir1/bad_files' as the 
     // corrupted files storage:
-    base.setWritable(false);
+    FileUtil.setWritable(base, false);
     
     FSDataInputStream dataFsdis = fileSys.open(dataPath);
     FSDataInputStream checksumFsdis = fileSys.open(checksumPath);
@@ -363,4 +348,23 @@ public class TestLocalFileSystem {
     assertTrue(checksumFileFound);
   }
   
+  @Test(timeout = 1000)
+  public void testSetTimes() throws Exception {
+    Path path = new Path(TEST_ROOT_DIR, "set-times");
+    writeFile(fileSys, path, 1);
+
+    // test only to the nearest second, as the raw FS may not
+    // support millisecond timestamps
+    long newModTime = 12345000;
+
+    FileStatus status = fileSys.getFileStatus(path);
+    assertTrue("check we're actually changing something", newModTime != status.getModificationTime());
+    assertEquals(0, status.getAccessTime());
+
+    fileSys.setTimes(path, newModTime, -1);
+    status = fileSys.getFileStatus(path);
+    assertEquals(newModTime, status.getModificationTime());
+    assertEquals(0, status.getAccessTime());
+}
+
 }

+ 135 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCopyPreserveFlag.java

@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.shell;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.shell.CopyCommands.Cp;
+import org.apache.hadoop.fs.shell.CopyCommands.Get;
+import org.apache.hadoop.fs.shell.CopyCommands.Put;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCopyPreserveFlag {
+  private static final int MODIFICATION_TIME = 12345000;
+  private static final Path FROM = new Path("d1", "f1");
+  private static final Path TO = new Path("d2", "f2");
+  private static final FsPermission PERMISSIONS = new FsPermission(
+    FsAction.ALL,
+    FsAction.EXECUTE,
+    FsAction.READ_WRITE);
+
+  private FileSystem fs;
+  private Path testDir;
+  private Configuration conf;
+
+  @Before
+  public void initialize() throws Exception {
+    conf = new Configuration(false);
+    conf.set("fs.file.impl", LocalFileSystem.class.getName());
+    fs = FileSystem.getLocal(conf);
+    testDir = new Path(
+        System.getProperty("test.build.data", "build/test/data") + "/testStat"
+    );
+    // don't want scheme on the path, just an absolute path
+    testDir = new Path(fs.makeQualified(testDir).toUri().getPath());
+
+    FileSystem.setDefaultUri(conf, fs.getUri());
+    fs.setWorkingDirectory(testDir);
+    fs.mkdirs(new Path("d1"));
+    fs.mkdirs(new Path("d2"));
+    fs.createNewFile(FROM);
+
+    FSDataOutputStream output = fs.create(FROM, true);
+    for(int i = 0; i < 100; ++i) {
+        output.writeInt(i);
+        output.writeChar('\n');
+    }
+    output.close();
+    fs.setTimes(FROM, MODIFICATION_TIME, 0);
+    fs.setPermission(FROM, PERMISSIONS);
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    fs.delete(testDir, true);
+    fs.close();
+  }
+
+  private void assertAttributesPreserved() throws IOException {
+    assertEquals(MODIFICATION_TIME, fs.getFileStatus(TO).getModificationTime());
+    assertEquals(PERMISSIONS, fs.getFileStatus(TO).getPermission());
+  }
+
+  private void assertAttributesChanged() throws IOException {
+      assertTrue(MODIFICATION_TIME != fs.getFileStatus(TO).getModificationTime());
+      assertTrue(!PERMISSIONS.equals(fs.getFileStatus(TO).getPermission()));
+  }
+
+  private void run(CommandWithDestination cmd, String... args) {
+    cmd.setConf(conf);
+    assertEquals(0, cmd.run(args));
+  }
+
+  @Test(timeout = 1000)
+  public void testPutWithP() throws Exception {
+    run(new Put(), "-p", FROM.toString(), TO.toString());
+    assertAttributesPreserved();
+  }
+
+  @Test(timeout = 1000)
+  public void testPutWithoutP() throws Exception {
+    run(new Put(), FROM.toString(), TO.toString());
+    assertAttributesChanged();
+  }
+
+  @Test(timeout = 1000)
+  public void testGetWithP() throws Exception {
+    run(new Get(), "-p", FROM.toString(), TO.toString());
+    assertAttributesPreserved();
+  }
+
+  @Test(timeout = 1000)
+  public void testGetWithoutP() throws Exception {
+    run(new Get(), FROM.toString(), TO.toString());
+    assertAttributesChanged();
+  }
+
+  @Test(timeout = 1000)
+  public void testCpWithP() throws Exception {
+      run(new Cp(), "-p", FROM.toString(), TO.toString());
+      assertAttributesPreserved();
+  }
+
+  @Test(timeout = 1000)
+  public void testCpWithoutP() throws Exception {
+      run(new Cp(), FROM.toString(), TO.toString());
+      assertAttributesChanged();
+  }
+}

+ 13 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ha/TestNodeFencer.java

@@ -24,6 +24,7 @@ import java.util.List;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.util.Shell;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -33,7 +34,12 @@ import com.google.common.collect.Lists;
 public class TestNodeFencer {
 
   private HAServiceTarget MOCK_TARGET;
-  
+
+  // Fencer shell commands that always return true on Unix and Windows
+  // respectively. Lacking the POSIX 'true' command on Windows, we use
+  // the batch command 'rem'.
+  private static String FENCER_TRUE_COMMAND_UNIX = "shell(true)";
+  private static String FENCER_TRUE_COMMAND_WINDOWS = "shell(rem)";
 
   @Before
   public void clearMockState() {
@@ -48,6 +54,11 @@ public class TestNodeFencer {
         .when(MOCK_TARGET).getAddress();
   }
 
+  private static String getFencerTrueCommand() {
+    return Shell.WINDOWS ?
+        FENCER_TRUE_COMMAND_WINDOWS : FENCER_TRUE_COMMAND_UNIX;
+  }
+
   @Test
   public void testSingleFencer() throws BadFencingConfigurationException {
     NodeFencer fencer = setupFencer(
@@ -100,7 +111,7 @@ public class TestNodeFencer {
 
   @Test
   public void testShortNameShell() throws BadFencingConfigurationException {
-    NodeFencer fencer = setupFencer("shell(true)");
+    NodeFencer fencer = setupFencer(getFencerTrueCommand());
     assertTrue(fencer.fence(MOCK_TARGET));
   }
 

+ 38 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/nativeio/TestNativeIO.java

@@ -240,6 +240,44 @@ public class TestNativeIO {
 
   }
 
+  /** Validate access checks on Windows */
+  @Test (timeout = 30000)
+  public void testAccess() throws Exception {
+    if (!Path.WINDOWS) {
+      return;
+    }
+
+    File testFile = new File(TEST_DIR, "testfileaccess");
+    assertTrue(testFile.createNewFile());
+
+    // Validate ACCESS_READ
+    FileUtil.setReadable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    FileUtil.setReadable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_READ));
+
+    // Validate ACCESS_WRITE
+    FileUtil.setWritable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    FileUtil.setWritable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_WRITE));
+
+    // Validate ACCESS_EXECUTE
+    FileUtil.setExecutable(testFile, false);
+    assertFalse(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+
+    FileUtil.setExecutable(testFile, true);
+    assertTrue(NativeIO.Windows.access(testFile.getAbsolutePath(),
+        NativeIO.Windows.AccessRight.ACCESS_EXECUTE));
+  }
+
   @Test (timeout = 30000)
   public void testOpenMissingWithoutCreate() throws Exception {
     if (Path.WINDOWS) {

+ 2 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/unix/TemporarySocketDirectory.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.util.Random;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.fs.FileUtil;
 
 /**
  * Create a temporary directory in which sockets can be created.
@@ -37,7 +38,7 @@ public class TemporarySocketDirectory implements Closeable {
     dir = new File(tmp, "socks." + (System.currentTimeMillis() +
         "." + (new Random().nextInt())));
     dir.mkdirs();
-    dir.setWritable(true, true);
+    FileUtil.setWritable(dir, true);
   }
 
   public File getDir() {

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

@@ -76,7 +76,9 @@ public class TestUserGroupInformation {
     javax.security.auth.login.Configuration.setConfiguration(
         new DummyLoginConfiguration());
     // doesn't matter what it is, but getGroups needs it set...
-    System.setProperty("hadoop.home.dir", "/tmp");
+    // use HADOOP_HOME environment variable to prevent interfering with logic
+    // that finds winutils.exe
+    System.setProperty("hadoop.home.dir", System.getenv("HADOOP_HOME"));
     // fake the realm is kerberos is enabled
     System.setProperty("java.security.krb5.kdc", "");
     System.setProperty("java.security.krb5.realm", "DEFAULT.REALM");

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java

@@ -28,6 +28,8 @@ import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 
+import org.apache.hadoop.fs.FileUtil;
+
 public class TestShell extends TestCase {
 
   private static class Command extends Shell {
@@ -92,7 +94,7 @@ public class TestShell extends TestCase {
     PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));
     writer.println(timeoutCommand);
     writer.close();
-    shellFile.setExecutable(true);
+    FileUtil.setExecutable(shellFile, true);
     Shell.ShellCommandExecutor shexc 
     = new Shell.ShellCommandExecutor(new String[]{shellFile.getAbsolutePath()},
                                       null, null, 100);

+ 42 - 10
hadoop-common-project/hadoop-common/src/test/resources/testConf.xml

@@ -133,7 +133,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
+          <expected-output>^-get( )*\[-p\]( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -141,7 +141,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*files, the destination must be a directory.( )*</expected-output>
+          <expected-output>^( |\t)*files, the destination must be a directory.( )*Passing( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*-p preserves access and modification times,( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*ownership and the mode.( )*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -276,7 +284,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-cp &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
+          <expected-output>^-cp \[-f\] \[-p\] &lt;src&gt; \.\.\. &lt;dst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt; to a( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -284,7 +292,11 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*must be a directory.( )*</expected-output>
+          <expected-output>^( |\t)*must be a directory.( )*Passing -p preserves access and( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*modification times, ownership and the mode.( )*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -372,11 +384,23 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-put &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Copy files from the local file system</expected-output>
+          <expected-output>^-put \[-f\] \[-p\] &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Copy files from the local file system</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*into fs.( )*Copying fails if the file already( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*into fs.( )*</expected-output>
+          <expected-output>^( |\t)*exists, unless the -f flag is given.( )*Passing( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*-p preserves access and modification times,( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*ownership and the mode.( )*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -391,7 +415,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-copyFromLocal &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Identical to the -put command\.</expected-output>
+          <expected-output>^-copyFromLocal \[-f\] \[-p\] &lt;localsrc&gt; \.\.\. &lt;dst&gt;:\s+Identical to the -put command\.</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -426,7 +450,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-get( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
+          <expected-output>^-get( )*\[-p\]( )*\[-ignoreCrc\]( )*\[-crc\]( )*&lt;src&gt; \.\.\. &lt;localdst&gt;:( |\t)*Copy files that match the file pattern &lt;src&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
@@ -434,7 +458,15 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*files, the destination must be a directory.( )*</expected-output>
+          <expected-output>^( |\t)*files, the destination must be a directory.( )*Passing( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*-p preserves access and modification times,( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*ownership and the mode.( )*</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -512,7 +544,7 @@
       <comparators>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^-copyToLocal \[-ignoreCrc\] \[-crc\] &lt;src&gt; \.\.\. &lt;localdst&gt;:\s+Identical to the -get command.</expected-output>
+          <expected-output>^-copyToLocal \[-p\] \[-ignoreCrc\] \[-crc\] &lt;src&gt; \.\.\. &lt;localdst&gt;:\s+Identical to the -get command.</expected-output>
         </comparator>
       </comparators>
     </test>

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -193,7 +193,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
      * Constructor.
      */
     public DoAsParam() {
-      super(NAME, null, UserProvider.USER_PATTERN);
+      super(NAME, null, UserProvider.getUserPattern());
     }
 
     /**
@@ -248,7 +248,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
      * Constructor.
      */
     public GroupParam() {
-      super(NAME, null, UserProvider.USER_PATTERN);
+      super(NAME, null, UserProvider.getUserPattern());
     }
 
   }
@@ -344,7 +344,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
      * Constructor.
      */
     public OwnerParam() {
-      super(NAME, null, UserProvider.USER_PATTERN);
+      super(NAME, null, UserProvider.getUserPattern());
     }
 
   }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServerWebApp.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.lib.server.ServerException;
 import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.servlet.ServerWebApp;
+import org.apache.hadoop.lib.wsrs.UserProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -102,6 +103,9 @@ public class HttpFSServerWebApp extends ServerWebApp {
     LOG.info("Connects to Namenode [{}]",
              get().get(FileSystemAccess.class).getFileSystemConfiguration().
                get(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY));
+    String userPattern = getConfig().get(UserProvider.USER_PATTERN_KEY, 
+      UserProvider.USER_PATTERN_DEFAULT);
+    UserProvider.setUserPattern(userPattern);
   }
 
   /**

+ 17 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/wsrs/UserProvider.java

@@ -41,12 +41,27 @@ public class UserProvider extends AbstractHttpContextInjectable<Principal> imple
 
   public static final String USER_NAME_PARAM = "user.name";
 
-  public static final Pattern USER_PATTERN = Pattern.compile("^[A-Za-z_][A-Za-z0-9._-]*[$]?$");
+
+  public static final String USER_PATTERN_KEY 
+    = "httpfs.user.provider.user.pattern";
+
+  public static final String USER_PATTERN_DEFAULT 
+    = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
+
+  private static Pattern userPattern = Pattern.compile(USER_PATTERN_DEFAULT);
+
+  public static void setUserPattern(String pattern) {
+    userPattern = Pattern.compile(pattern);
+  }
+
+  public static Pattern getUserPattern() {
+    return userPattern;
+  }
 
   static class UserParam extends StringParam {
 
     public UserParam(String user) {
-      super(USER_NAME_PARAM, user, USER_PATTERN);
+      super(USER_NAME_PARAM, user, getUserPattern());
     }
 
     @Override

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml

@@ -226,4 +226,12 @@
     </description>
   </property>
 
+  <property>
+    <name>httpfs.user.provider.user.pattern</name>
+    <value>^[A-Za-z_][A-Za-z0-9._-]*[$]?$</value>
+    <description>
+      Valid pattern for user and group names, it must be a valid java regex.
+    </description>
+  </property>
+
 </configuration>

+ 94 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSCustomUserName.java

@@ -0,0 +1,94 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.http.server;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
+import org.apache.hadoop.lib.server.Service;
+import org.apache.hadoop.lib.server.ServiceException;
+import org.apache.hadoop.lib.service.Groups;
+import org.apache.hadoop.lib.wsrs.UserProvider;
+import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
+import org.apache.hadoop.security.authentication.server.AuthenticationToken;
+import org.apache.hadoop.security.authentication.util.Signer;
+import org.apache.hadoop.test.HFSTestCase;
+import org.apache.hadoop.test.HadoopUsersConfTestHelper;
+import org.apache.hadoop.test.TestDir;
+import org.apache.hadoop.test.TestDirHelper;
+import org.apache.hadoop.test.TestHdfs;
+import org.apache.hadoop.test.TestHdfsHelper;
+import org.apache.hadoop.test.TestJetty;
+import org.apache.hadoop.test.TestJettyHelper;
+import org.json.simple.JSONObject;
+import org.json.simple.parser.JSONParser;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mortbay.jetty.Server;
+import org.mortbay.jetty.webapp.WebAppContext;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.Writer;
+import java.net.HttpURLConnection;
+import java.net.URL;
+import java.text.MessageFormat;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestHttpFSCustomUserName extends HFSTestCase {
+
+  @Test
+  @TestDir
+  @TestJetty
+  public void defaultUserName() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+
+    Configuration httpfsConf = new Configuration(false);
+    HttpFSServerWebApp server = 
+      new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    server.init();
+    Assert.assertEquals(UserProvider.USER_PATTERN_DEFAULT, 
+      UserProvider.getUserPattern().pattern());
+    server.destroy();
+  }
+
+  @Test
+  @TestDir
+  @TestJetty
+  public void customUserName() throws Exception {
+    String dir = TestDirHelper.getTestDir().getAbsolutePath();
+
+    Configuration httpfsConf = new Configuration(false);
+    httpfsConf.set(UserProvider.USER_PATTERN_KEY, "1");
+    HttpFSServerWebApp server =
+      new HttpFSServerWebApp(dir, dir, dir, dir, httpfsConf);
+    server.init();
+    Assert.assertEquals("1", UserProvider.getUserPattern().pattern());
+    server.destroy();
+  }
+
+}

+ 15 - 10
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/wsrs/TestUserProvider.java

@@ -104,34 +104,39 @@ public class TestUserProvider {
   @Test
   @TestException(exception = IllegalArgumentException.class)
   public void userNameEmpty() {
-    UserProvider.UserParam userParam = new UserProvider.UserParam("username");
-    userParam.parseParam("");
+    new UserProvider.UserParam("");
   }
 
   @Test
   @TestException(exception = IllegalArgumentException.class)
   public void userNameInvalidStart() {
-    UserProvider.UserParam userParam = new UserProvider.UserParam("username");
-    userParam.parseParam("1x");
+    new UserProvider.UserParam("1x");
   }
 
   @Test
   @TestException(exception = IllegalArgumentException.class)
   public void userNameInvalidDollarSign() {
-    UserProvider.UserParam userParam = new UserProvider.UserParam("username");
-    userParam.parseParam("1$x");
+    new UserProvider.UserParam("1$x");
   }
 
   @Test
   public void userNameMinLength() {
-    UserProvider.UserParam userParam = new UserProvider.UserParam("username");
-    assertNotNull(userParam.parseParam("a"));
+    new UserProvider.UserParam("a");
   }
 
   @Test
   public void userNameValidDollarSign() {
-    UserProvider.UserParam userParam = new UserProvider.UserParam("username");
-    assertNotNull(userParam.parseParam("a$"));
+    new UserProvider.UserParam("a$");
+  }
+
+  @Test
+  public void customUserPattern() {
+    try {
+      UserProvider.setUserPattern("1");
+      new UserProvider.UserParam("1");      
+    } finally {
+      UserProvider.setUserPattern(UserProvider.USER_PATTERN_DEFAULT);
+    }
   }
 
 }

+ 42 - 4
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -22,9 +22,6 @@ Trunk (Unreleased)
     Azure environments. (See breakdown of tasks below for subtasks and
     contributors)
 
-    HDFS-2576. Enhances the DistributedFileSystem's create API so that clients
-    can specify favored datanodes for a file's blocks. (ddas)
-
   IMPROVEMENTS
 
     HDFS-4665. Move TestNetworkTopologyWithNodeGroup to common.
@@ -268,7 +265,10 @@ Trunk (Unreleased)
     HDFS-4761. When resetting FSDirectory, the inodeMap should also be reset.
     (Jing Zhao via szetszwo)
 
-  BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
+    HDFS-4687. TestDelegationTokenForProxyUser#testWebHdfsDoAs is flaky with
+    JDK7. (Andrew Wang via atm)
+
+  BREAKDOWN OF HADOOP-8562 and HDFS-3602 SUBTASKS AND RELATED JIRAS
 
     HDFS-4145. Merge hdfs cmd line scripts from branch-1-win. (David Lao,
     Bikas Saha, Lauren Yang, Chuan Liu, Thejas M Nair and Ivan Mitic via suresh)
@@ -311,6 +311,27 @@ Trunk (Unreleased)
     HDFS-4584. Skip TestNNWithQJM.testNewNamenodeTakesOverWriter() on Windows.
     (Arpit Agarwal via szetszwo)
 
+    HDFS-4741. TestStorageRestore#testStorageRestoreFailure fails on Windows.
+    (Arpit Agarwal via suresh)
+
+    HDFS-4743. TestNNStorageRetentionManager fails on Windows.
+    (Chris Nauroth via suresh)
+
+    HDFS-4740. Fixes for a few test failures on Windows.
+    (Arpit Agarwal via suresh)
+
+    HDFS-4722. TestGetConf#testFederation times out on Windows.
+    (Ivan Mitic via suresh)
+
+    HDFS-4705. Address HDFS test failures on Windows because of invalid
+    dfs.namenode.name.dir. (Ivan Mitic via suresh)
+
+    HDFS-4734. HDFS Tests that use ShellCommandFencer are broken on Windows.
+    (Arpit Agarwal via suresh)
+
+    HDFS-4610. Use common utils FileUtil#setReadable/Writable/Executable and 
+    FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
@@ -373,6 +394,9 @@ Release 2.0.5-beta - UNRELEASED
 
     HDFS-4434. Provide a mapping from INodeId to INode. (suresh)
 
+    HDFS-4305. Add a configurable limit on number of blocks per file, and min
+    block size. (Andrew Wang via atm)
+
   NEW FEATURES
 
     HDFS-1804. Add a new block-volume device choosing policy that looks at
@@ -391,6 +415,9 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
     exists.  (Brandon Li via suresh)
 
+    HDFS-2576. Enhances the DistributedFileSystem's create API so that clients
+    can specify favored datanodes for a file's blocks. (ddas)
+
   IMPROVEMENTS
 
     HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
@@ -582,6 +609,17 @@ Release 2.0.5-beta - UNRELEASED
     HDFS-4745. TestDataTransferKeepalive#testSlowReader has race condition that
     causes sporadic failure. (Chris Nauroth via suresh)
 
+    HDFS-4768. File handle leak in datanode when a block pool is removed.
+    (Chris Nauroth via suresh)
+
+    HDFS-4748. MiniJournalCluster#restartJournalNode leaks resources, which 
+    causes sporadic test failures. (Chris Nauroth via suresh)
+
+    HDFS-4733. Make HttpFS username pattern configurable. (tucu via atm)
+
+    HDFS-4778. Fixes some issues that the first patch on HDFS-2576 missed.
+    (ddas)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/pom.xml

@@ -33,6 +33,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <properties>
     <hadoop.component>hdfs</hadoop.component>
+    <hadoop.common.build.dir>${basedir}/../../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
 
   <dependencies>

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -227,6 +227,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_MAX_COMPONENT_LENGTH_DEFAULT = 0; // no limit
   public static final String  DFS_NAMENODE_MAX_DIRECTORY_ITEMS_KEY = "dfs.namenode.fs-limits.max-directory-items";
   public static final int     DFS_NAMENODE_MAX_DIRECTORY_ITEMS_DEFAULT = 0; // no limit
+  public static final String  DFS_NAMENODE_MIN_BLOCK_SIZE_KEY = "dfs.namenode.fs-limits.min-block-size";
+  public static final long    DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT = 1024*1024;
+  public static final String  DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY = "dfs.namenode.fs-limits.max-blocks-per-file";
+  public static final long    DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT = 1024*1024;
 
   //Following keys have no defaults
   public static final String  DFS_DATANODE_DATA_DIR_KEY = "dfs.datanode.data.dir";

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java

@@ -170,7 +170,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
           results.add(remainingTargets[i]);
         }
       }
-      return results.toArray(new DatanodeDescriptor[results.size()]);
+      return getPipeline(writer,
+          results.toArray(new DatanodeDescriptor[results.size()]));
     } catch (NotEnoughReplicasException nr) {
       // Fall back to regular block placement disregarding favored nodes hint
       return chooseTarget(src, numOfReplicas, writer, 

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java

@@ -339,7 +339,6 @@ public class DatanodeManager {
    *
    * @param address hostaddress:transfer address
    * @return the best match for the given datanode
-   * @throws IOException when no datanode is found for given address
    */
   DatanodeDescriptor getDatanodeDescriptor(String address) {
     DatanodeDescriptor node = null;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -448,7 +448,7 @@ public abstract class Storage extends StorageInfo {
           LOG.warn(rootPath + "is not a directory");
           return StorageState.NON_EXISTENT;
         }
-        if (!root.canWrite()) {
+        if (!FileUtil.canWrite(root)) {
           LOG.warn("Cannot access storage directory " + rootPath);
           return StorageState.NON_EXISTENT;
         }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -267,7 +267,10 @@ public class DataBlockScanner implements Runnable {
   }
   
   public synchronized void removeBlockPool(String blockPoolId) {
-    blockPoolScannerMap.remove(blockPoolId);
+    BlockPoolSliceScanner bpss = blockPoolScannerMap.remove(blockPoolId);
+    if (bpss != null) {
+      bpss.shutdown();
+    }
     LOG.info("Removed bpid="+blockPoolId+" from blockPoolScannerMap");
   }
   

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java

@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -128,7 +129,7 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
   static long readCheckpointTime(StorageDirectory sd) throws IOException {
     File timeFile = NNStorage.getStorageFile(sd, NameNodeFile.TIME);
     long timeStamp = 0L;
-    if (timeFile.exists() && timeFile.canRead()) {
+    if (timeFile.exists() && FileUtil.canRead(timeFile)) {
       DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
       try {
         timeStamp = in.readLong();

+ 18 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -373,6 +373,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   private final long maxFsObjects;          // maximum number of fs objects
 
+  private final long minBlockSize;         // minimum block size
+  private final long maxBlocksPerFile;     // maximum # of blocks per file
+
   /**
    * The global generation stamp for this file system. 
    */
@@ -604,6 +607,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.maxFsObjects = conf.getLong(DFS_NAMENODE_MAX_OBJECTS_KEY, 
                                        DFS_NAMENODE_MAX_OBJECTS_DEFAULT);
 
+      this.minBlockSize = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT);
+      this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY,
+          DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT);
       this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
           DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
       this.supportAppends = conf.getBoolean(DFS_SUPPORT_APPEND_KEY, DFS_SUPPORT_APPEND_DEFAULT);
@@ -1838,6 +1845,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     final HdfsFileStatus stat;
     FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
+    if (blockSize < minBlockSize) {
+      throw new IOException("Specified block size is less than configured" +
+          " minimum value (" + DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY
+          + "): " + blockSize + " < " + minBlockSize);
+    }
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     writeLock();
     try {
@@ -2260,7 +2272,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         // This is a retry. Just return the last block.
         return onRetryBlock[0];
       }
-
+      if (pendingFile.getBlocks().length >= maxBlocksPerFile) {
+        throw new IOException("File has reached the limit on maximum number of"
+            + " blocks (" + DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY
+            + "): " + pendingFile.getBlocks().length + " >= "
+            + maxBlocksPerFile);
+      }
       blockSize = pendingFile.getPreferredBlockSize();
       clientNode = pendingFile.getClientNode();
       replication = pendingFile.getFileReplication();

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -34,6 +34,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
@@ -230,8 +231,8 @@ public class NNStorage extends Storage implements Closeable,
         File root = sd.getRoot();
         LOG.info("currently disabled dir " + root.getAbsolutePath() +
                  "; type="+sd.getStorageDirType() 
-                 + ";canwrite="+root.canWrite());
-        if(root.exists() && root.canWrite()) {
+                 + ";canwrite="+FileUtil.canWrite(root));
+        if(root.exists() && FileUtil.canWrite(root)) {
           LOG.info("restoring dir " + sd.getRoot().getAbsolutePath());
           this.addStorageDir(sd); // restore
           this.removedStorageDirs.remove(sd);
@@ -505,7 +506,7 @@ public class NNStorage extends Storage implements Closeable,
       dirIterator(NameNodeDirType.IMAGE); it.hasNext();) {
       sd = it.next();
       File fsImage = getStorageFile(sd, NameNodeFile.IMAGE, txid);
-      if(sd.getRoot().canRead() && fsImage.exists())
+      if(FileUtil.canRead(sd.getRoot()) && fsImage.exists())
         return fsImage;
     }
     return null;
@@ -722,7 +723,7 @@ public class NNStorage extends Storage implements Closeable,
   private File findFile(NameNodeDirType dirType, String name) {
     for (StorageDirectory sd : dirIterable(dirType)) {
       File candidate = new File(sd.getCurrentDir(), name);
-      if (sd.getCurrentDir().canRead() &&
+      if (FileUtil.canRead(sd.getCurrentDir()) &&
           candidate.exists()) {
         return candidate;
       }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java

@@ -33,6 +33,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.http.HttpConfig;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.Time;
@@ -102,7 +103,7 @@ public class TransferFsImage {
     assert !dstFiles.isEmpty() : "No checkpoint targets.";
     
     for (File f : dstFiles) {
-      if (f.exists() && f.canRead()) {
+      if (f.exists() && FileUtil.canRead(f)) {
         LOG.info("Skipping download of remote edit log " +
             log + " since it already is stored locally at " + f);
         return;

+ 17 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -238,6 +238,23 @@
       contain.  A value of 0 will disable the check.</description>
 </property>
 
+<property>
+  <name>dfs.namenode.fs-limits.min-block-size</name>
+  <value>1048576</value>
+  <description>Minimum block size in bytes, enforced by the Namenode at create
+      time. This prevents the accidental creation of files with tiny block
+      sizes (and thus many blocks), which can degrade
+      performance.</description>
+</property>
+
+<property>
+    <name>dfs.namenode.fs-limits.max-blocks-per-file</name>
+    <value>1048576</value>
+    <description>Maximum number of blocks per file, enforced by the Namenode on
+        write. This prevents the creation of extremely large files which can
+        degrade performance.</description>
+</property>
+
 <property>
   <name>dfs.namenode.edits.dir</name>
   <value>${dfs.namenode.name.dir}</value>

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

@@ -681,9 +681,9 @@ public class MiniDFSCluster {
       sb.append("\tabsolute:").append(path.getAbsolutePath()).append("\n");
       sb.append("\tpermissions: ");
       sb.append(path.isDirectory() ? "d": "-");
-      sb.append(path.canRead() ? "r" : "-");
-      sb.append(path.canWrite() ? "w" : "-");
-      sb.append(path.canExecute() ? "x" : "-");
+      sb.append(FileUtil.canRead(path) ? "r" : "-");
+      sb.append(FileUtil.canWrite(path) ? "w" : "-");
+      sb.append(FileUtil.canExecute(path) ? "x" : "-");
       sb.append("\n");
       path = path.getParentFile();
     }

+ 16 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java

@@ -29,11 +29,11 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_A
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICES;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMESERVICE_ID;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import org.apache.hadoop.util.Shell;
+
+import static org.junit.Assert.*;
+import org.junit.Assume;
+import static org.hamcrest.CoreMatchers.*;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
@@ -619,19 +619,25 @@ public class TestDFSUtil {
     
     assertEquals(1, uris.size());
     assertTrue(uris.contains(new URI("hdfs://" + NN1_SRVC_ADDR)));
+  }
+
+  @Test (timeout=15000)
+  public void testLocalhostReverseLookup() {
+    // 127.0.0.1 -> localhost reverse resolution does not happen on Windows.
+    Assume.assumeTrue(!Shell.WINDOWS);
 
     // Make sure when config FS_DEFAULT_NAME_KEY using IP address,
     // it will automatically convert it to hostname
-    conf = new HdfsConfiguration();
+    HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY, "hdfs://127.0.0.1:8020");
-    uris = DFSUtil.getNameServiceUris(conf);
+    Collection<URI> uris = DFSUtil.getNameServiceUris(conf);
     assertEquals(1, uris.size());
     for (URI uri : uris) {
-      assertFalse(uri.getHost().equals("127.0.0.1"));
+      assertThat(uri.getHost(), not("127.0.0.1"));
     }
   }
-  
-  @Test
+
+  @Test (timeout=15000)
   public void testIsValidName() {
     assertFalse(DFSUtil.isValidName("/foo/../bar"));
     assertFalse(DFSUtil.isValidName("/foo/./bar"));

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLargeBlock.java

@@ -155,10 +155,12 @@ public class TestLargeBlock {
   }
  
   /**
-   * Test for block size of 2GB + 512B
+   * Test for block size of 2GB + 512B. This test can take a rather long time to
+   * complete on Windows (reading the file back can be slow) so we use a larger
+   * timeout here.
    * @throws IOException in case of errors
    */
-  @Test(timeout = 120000)
+  @Test (timeout = 900000)
   public void testLargeBlockSize() throws IOException {
     final long blockSize = 2L * 1024L * 1024L * 1024L + 512L; // 2GB + 512B
     runTest(blockSize);

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/MiniJournalCluster.java

@@ -178,9 +178,9 @@ public class MiniJournalCluster {
     conf.set(DFSConfigKeys.DFS_JOURNALNODE_HTTP_ADDRESS_KEY, "127.0.0.1:" +
         httpAddrs[i].getPort());
     
-    JournalNode jn = new JournalNode();
-    jn.setConf(conf);
-    jn.start();
+    nodes[i] = new JournalNode();
+    nodes[i].setConf(conf);
+    nodes[i].start();
   }
 
   public int getQuorumSize() {

+ 16 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java

@@ -60,24 +60,26 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.token.Token;
 import org.apache.log4j.Level;
-import org.junit.After;
+import org.junit.AfterClass;
 import org.junit.Assert;
-import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 public class TestDelegationTokenForProxyUser {
-  private MiniDFSCluster cluster;
-  Configuration config;
+  private static MiniDFSCluster cluster;
+  private static Configuration config;
   final private static String GROUP1_NAME = "group1";
   final private static String GROUP2_NAME = "group2";
   final private static String[] GROUP_NAMES = new String[] { GROUP1_NAME,
       GROUP2_NAME };
   final private static String REAL_USER = "RealUser";
   final private static String PROXY_USER = "ProxyUser";
+  private static UserGroupInformation ugi;
+  private static UserGroupInformation proxyUgi;
   
   private static final Log LOG = LogFactory.getLog(TestDoAsEffectiveUser.class);
   
-  private void configureSuperUserIPAddresses(Configuration conf,
+  private static void configureSuperUserIPAddresses(Configuration conf,
       String superUserShortName) throws IOException {
     ArrayList<String> ipList = new ArrayList<String>();
     Enumeration<NetworkInterface> netInterfaceList = NetworkInterface
@@ -102,8 +104,8 @@ public class TestDelegationTokenForProxyUser {
         builder.toString());
   }
   
-  @Before
-  public void setUp() throws Exception {
+  @BeforeClass
+  public static void setUp() throws Exception {
     config = new HdfsConfiguration();
     config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     config.setLong(
@@ -119,21 +121,20 @@ public class TestDelegationTokenForProxyUser {
     cluster = new MiniDFSCluster.Builder(config).build();
     cluster.waitActive();
     ProxyUsers.refreshSuperUserGroupsConfiguration(config);
+    ugi = UserGroupInformation.createRemoteUser(REAL_USER);
+    proxyUgi = UserGroupInformation.createProxyUserForTesting(PROXY_USER, ugi,
+        GROUP_NAMES);
   }
 
-  @After
-  public void tearDown() throws Exception {
+  @AfterClass
+  public static void tearDown() throws Exception {
     if(cluster!=null) {
       cluster.shutdown();
     }
   }
  
-  @Test
+  @Test(timeout=20000)
   public void testDelegationTokenWithRealUser() throws IOException {
-    UserGroupInformation ugi = UserGroupInformation
-        .createRemoteUser(REAL_USER);
-    final UserGroupInformation proxyUgi = UserGroupInformation
-        .createProxyUserForTesting(PROXY_USER, ugi, GROUP_NAMES);
     try {
       Token<?>[] tokens = proxyUgi
           .doAs(new PrivilegedExceptionAction<Token<?>[]>() {
@@ -154,12 +155,11 @@ public class TestDelegationTokenForProxyUser {
     }
   }
   
-  @Test
+  @Test(timeout=20000)
   public void testWebHdfsDoAs() throws Exception {
     WebHdfsTestUtil.LOG.info("START: testWebHdfsDoAs()");
     ((Log4JLogger)NamenodeWebHdfsMethods.LOG).getLogger().setLevel(Level.ALL);
     ((Log4JLogger)ExceptionHandler.LOG).getLogger().setLevel(Level.ALL);
-    final UserGroupInformation ugi = UserGroupInformation.createRemoteUser(REAL_USER);
     WebHdfsTestUtil.LOG.info("ugi.getShortUserName()=" + ugi.getShortUserName());
     final WebHdfsFileSystem webhdfs = WebHdfsTestUtil.getWebHdfsFileSystemAs(ugi, config);
     

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java

@@ -31,6 +31,7 @@ import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockReader;
 import org.apache.hadoop.hdfs.BlockReaderFactory;
@@ -91,10 +92,10 @@ public class TestDataNodeVolumeFailure {
   @After
   public void tearDown() throws Exception {
     if(data_fail != null) {
-      data_fail.setWritable(true);
+      FileUtil.setWritable(data_fail, true);
     }
     if(failedDir != null) {
-      failedDir.setWritable(true);
+      FileUtil.setWritable(failedDir, true);
     }
     if(cluster != null) {
       cluster.shutdown();

+ 13 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java

@@ -31,6 +31,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -88,8 +89,8 @@ public class TestDataNodeVolumeFailureReporting {
   @After
   public void tearDown() throws Exception {
     for (int i = 0; i < 3; i++) {
-      new File(dataDir, "data"+(2*i+1)).setExecutable(true);
-      new File(dataDir, "data"+(2*i+2)).setExecutable(true);
+      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
+      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
     }
     cluster.shutdown();
   }
@@ -131,8 +132,8 @@ public class TestDataNodeVolumeFailureReporting {
      * fail. The client does not retry failed nodes even though
      * perhaps they could succeed because just a single volume failed.
      */
-    assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(false));
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
 
     /*
      * Create file1 and wait for 3 replicas (ie all DNs can still
@@ -168,7 +169,7 @@ public class TestDataNodeVolumeFailureReporting {
      * Now fail a volume on the third datanode. We should be able to get
      * three replicas since we've already identified the other failures.
      */
-    assertTrue("Couldn't chmod local vol", dn3Vol1.setExecutable(false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, false));
     Path file2 = new Path("/test2");
     DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file2, (short)3);
@@ -200,7 +201,7 @@ public class TestDataNodeVolumeFailureReporting {
      * and that it's no longer up. Only wait for two replicas since
      * we'll never get a third.
      */
-    assertTrue("Couldn't chmod local vol", dn3Vol2.setExecutable(false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, false));
     Path file3 = new Path("/test3");
     DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file3, (short)2);
@@ -222,10 +223,10 @@ public class TestDataNodeVolumeFailureReporting {
      * restart, so file creation should be able to succeed after
      * restoring the data directories and restarting the datanodes.
      */
-    assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(true));
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
-    assertTrue("Couldn't chmod local vol", dn3Vol1.setExecutable(true));
-    assertTrue("Couldn't chmod local vol", dn3Vol2.setExecutable(true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, true));
     cluster.restartDataNodes();
     cluster.waitActive();
     Path file4 = new Path("/test4");
@@ -261,8 +262,8 @@ public class TestDataNodeVolumeFailureReporting {
     // third healthy so one node in the pipeline will not fail). 
     File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    assertTrue("Couldn't chmod local vol", dn1Vol1.setExecutable(false));
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
 
     Path file1 = new Path("/test1");
     DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java

@@ -77,8 +77,8 @@ public class TestDataNodeVolumeFailureToleration {
   @After
   public void tearDown() throws Exception {
     for (int i = 0; i < 3; i++) {
-      new File(dataDir, "data"+(2*i+1)).setExecutable(true);
-      new File(dataDir, "data"+(2*i+2)).setExecutable(true);
+      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
+      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
     }
     cluster.shutdown();
   }
@@ -152,7 +152,7 @@ public class TestDataNodeVolumeFailureToleration {
 
     // Fail a volume on the 2nd DN
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(false));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
 
     // Should only get two replicas (the first DN and the 3rd)
     Path file1 = new Path("/test1");
@@ -165,7 +165,7 @@ public class TestDataNodeVolumeFailureToleration {
 
     // If we restore the volume we should still only be able to get
     // two replicas since the DN is still considered dead.
-    assertTrue("Couldn't chmod local vol", dn2Vol1.setExecutable(true));
+    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
     Path file2 = new Path("/test2");
     DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file2, (short)2);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDiskError.java

@@ -27,6 +27,7 @@ import java.net.Socket;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -106,8 +107,8 @@ public class TestDiskError {
       }
     } finally {
       // restore its old permission
-      dir1.setWritable(true);
-      dir2.setWritable(true);
+      FileUtil.setWritable(dir1, true);
+      FileUtil.setWritable(dir2, true);
     }
   }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAllowFormat.java

@@ -169,6 +169,8 @@ public class TestAllowFormat {
     InetSocketAddress nnAddr2 = new InetSocketAddress(localhost, 9020);
     HATestUtil.setFailoverConfigurations(conf, logicalName, nnAddr1, nnAddr2);
 
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY,
+        new File(hdfsDir, "name").getAbsolutePath());
     conf.setBoolean(DFS_NAMENODE_SUPPORT_ALLOW_FORMAT_KEY, true);
     conf.set(DFSUtil.addKeySuffixes(
         DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX, "dummy"),

+ 14 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -157,7 +157,7 @@ public class TestCheckpoint {
       
       try {
         // Simulate the mount going read-only
-        dir.setWritable(false);
+        FileUtil.setWritable(dir, false);
         cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
             .format(false).build();
         fail("NN should have failed to start with " + dir + " set unreadable");
@@ -167,7 +167,7 @@ public class TestCheckpoint {
       } finally {
         cleanup(cluster);
         cluster = null;
-        dir.setWritable(true);
+        FileUtil.setWritable(dir, true);
       }
     }
   }
@@ -750,9 +750,12 @@ public class TestCheckpoint {
   @Test
   public void testSeparateEditsDirLocking() throws IOException {
     Configuration conf = new HdfsConfiguration();
-    File editsDir = new File(MiniDFSCluster.getBaseDirectory() +
-        "/testSeparateEditsDirLocking");
-    
+    File nameDir = new File(MiniDFSCluster.getBaseDirectory(), "name");
+    File editsDir = new File(MiniDFSCluster.getBaseDirectory(),
+        "testSeparateEditsDirLocking");
+
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        nameDir.getAbsolutePath());
     conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
         editsDir.getAbsolutePath());
     MiniDFSCluster cluster = null;
@@ -1822,7 +1825,7 @@ public class TestCheckpoint {
       StorageDirectory sd1 = storage.getStorageDir(1);
       
       currentDir = sd0.getCurrentDir();
-      currentDir.setExecutable(false);
+      FileUtil.setExecutable(currentDir, false);
 
       // Upload checkpoint when NN has a bad storage dir. This should
       // succeed and create the checkpoint in the good dir.
@@ -1832,7 +1835,7 @@ public class TestCheckpoint {
           new File(sd1.getCurrentDir(), NNStorage.getImageFileName(2)));
       
       // Restore the good dir
-      currentDir.setExecutable(true);
+      FileUtil.setExecutable(currentDir, true);
       nn.restoreFailedStorage("true");
       nn.rollEditLog();
 
@@ -1843,7 +1846,7 @@ public class TestCheckpoint {
       assertParallelFilesInvariant(cluster, ImmutableList.of(secondary));
     } finally {
       if (currentDir != null) {
-        currentDir.setExecutable(true);
+        FileUtil.setExecutable(currentDir, true);
       }
       cleanup(secondary);
       secondary = null;
@@ -1893,7 +1896,7 @@ public class TestCheckpoint {
       StorageDirectory sd0 = storage.getStorageDir(0);
       assertEquals(NameNodeDirType.IMAGE, sd0.getStorageDirType());
       currentDir = sd0.getCurrentDir();
-      currentDir.setExecutable(false);
+      FileUtil.setExecutable(currentDir, false);
 
       // Try to upload checkpoint -- this should fail since there are no
       // valid storage dirs
@@ -1906,7 +1909,7 @@ public class TestCheckpoint {
       }
       
       // Restore the good dir
-      currentDir.setExecutable(true);
+      FileUtil.setExecutable(currentDir, true);
       nn.restoreFailedStorage("true");
       nn.rollEditLog();
 
@@ -1917,7 +1920,7 @@ public class TestCheckpoint {
       assertParallelFilesInvariant(cluster, ImmutableList.of(secondary));
     } finally {
       if (currentDir != null) {
-        currentDir.setExecutable(true);
+        FileUtil.setExecutable(currentDir, true);
       }
       cleanup(secondary);
       secondary = null;

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -881,14 +881,14 @@ public class TestEditLog {
     logDir.mkdirs();
     FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
     try {
-      logDir.setWritable(false);
+      FileUtil.setWritable(logDir, false);
       log.openForWrite();
       fail("Did no throw exception on only having a bad dir");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
           "too few journals successfully started", ioe);
     } finally {
-      logDir.setWritable(true);
+      FileUtil.setWritable(logDir, true);
       log.close();
     }
   }

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java

@@ -19,20 +19,30 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.junit.Assert.assertEquals;
 
+import java.io.File;
 import java.io.IOException;
 import java.net.URI;
 import java.util.Collection;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.junit.After;
 import org.junit.Test;
 
 public class TestFSNamesystem {
 
+  @After
+  public void cleanUp() {
+    FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
+  }
+
   /**
    * Tests that the namenode edits dirs are gotten with duplicates removed
    */
@@ -54,6 +64,9 @@ public class TestFSNamesystem {
   @Test
   public void testFSNamespaceClearLeases() throws Exception {
     Configuration conf = new HdfsConfiguration();
+    File nameDir = new File(MiniDFSCluster.getBaseDirectory(), "name");
+    conf.set(DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+
     NameNode.initMetrics(conf, NamenodeRole.NAMENODE);
     DFSTestUtil.formatNameNode(conf);
     FSNamesystem fsn = FSNamesystem.loadFromDisk(conf);

+ 16 - 16
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java

@@ -22,6 +22,7 @@ import static org.junit.Assert.*;
 
 import java.util.ArrayList;
 import java.util.Random;
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
@@ -32,6 +33,7 @@ import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -67,7 +69,7 @@ public class TestFavoredNodesEndToEnd {
     }
   }
 
-  @Test
+  @Test(timeout=180000)
   public void testFavoredNodesEndToEnd() throws Exception {
     //create 10 files with random preferred nodes
     for (int i = 0; i < NUM_FILES; i++) {
@@ -80,11 +82,7 @@ public class TestFavoredNodesEndToEnd {
           4096, (short)3, (long)4096, null, datanode);
       out.write(SOME_BYTES);
       out.close();
-      BlockLocation[] locations = 
-          dfs.getClient().getBlockLocations(p.toUri().getPath(), 0, 
-              Long.MAX_VALUE);
-      //make sure we have exactly one block location, and three hosts
-      assertTrue(locations.length == 1 && locations[0].getHosts().length == 3);
+      BlockLocation[] locations = getBlockLocations(p);
       //verify the files got created in the right nodes
       for (BlockLocation loc : locations) {
         String[] hosts = loc.getNames();
@@ -94,7 +92,7 @@ public class TestFavoredNodesEndToEnd {
     }
   }
 
-  @Test
+  @Test(timeout=180000)
   public void testWhenFavoredNodesNotPresent() throws Exception {
     //when we ask for favored nodes but the nodes are not there, we should
     //get some other nodes. In other words, the write to hdfs should not fail
@@ -110,13 +108,10 @@ public class TestFavoredNodesEndToEnd {
         4096, (short)3, (long)4096, null, arbitraryAddrs);
     out.write(SOME_BYTES);
     out.close();
-    BlockLocation[] locations = 
-        dfs.getClient().getBlockLocations(p.toUri().getPath(), 0, 
-            Long.MAX_VALUE);
-    assertTrue(locations.length == 1 && locations[0].getHosts().length == 3);
+    getBlockLocations(p);
   }
 
-  @Test
+  @Test(timeout=180000)
   public void testWhenSomeNodesAreNotGood() throws Exception {
     //make some datanode not "good" so that even if the client prefers it,
     //the namenode would not give it as a replica to write to
@@ -136,12 +131,9 @@ public class TestFavoredNodesEndToEnd {
         4096, (short)3, (long)4096, null, addrs);
     out.write(SOME_BYTES);
     out.close();
-    BlockLocation[] locations = 
-        dfs.getClient().getBlockLocations(p.toUri().getPath(), 0, 
-            Long.MAX_VALUE);
     //reset the state
     d.stopDecommission();
-    assertTrue(locations.length == 1 && locations[0].getHosts().length == 3);
+    BlockLocation[] locations = getBlockLocations(p);
     //also make sure that the datanode[0] is not in the list of hosts
     String datanode0 = 
         datanodes.get(0).getXferAddress().getAddress().getHostAddress()
@@ -153,6 +145,14 @@ public class TestFavoredNodesEndToEnd {
     }
   }
 
+  private BlockLocation[] getBlockLocations(Path p) throws Exception {
+    DFSTestUtil.waitReplication(dfs, p, (short)3);
+    BlockLocation[] locations = dfs.getClient().getBlockLocations(
+        p.toUri().getPath(), 0, Long.MAX_VALUE);
+    assertTrue(locations.length == 1 && locations[0].getHosts().length == 3);
+    return locations;
+  }
+
   private String[] getStringForInetSocketAddrs(InetSocketAddress[] datanode) {
     String strs[] = new String[datanode.length];
     for (int i = 0; i < datanode.length; i++) {

+ 58 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java

@@ -28,7 +28,10 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.Test;
 
 
@@ -159,4 +162,59 @@ public class TestFileLimit {
     testFileLimit();
     simulatedStorage = false;
   }
+
+  @Test(timeout=60000)
+  public void testMaxBlocksPerFileLimit() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    // Make a small block size and a low limit
+    final long blockSize = 4096;
+    final long numBlocks = 2;
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY, numBlocks);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    FileSystem fs = cluster.getFileSystem();
+    HdfsDataOutputStream fout =
+        (HdfsDataOutputStream)fs.create(new Path("/testmaxfilelimit"));
+    try {
+      // Write maximum number of blocks
+      fout.write(new byte[(int)blockSize*(int)numBlocks]);
+      fout.hflush();
+      // Try to write one more block
+      try {
+        fout.write(new byte[1]);
+        fout.hflush();
+        assert false : "Expected IOException after writing too many blocks";
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains("File has reached the limit" +
+            " on maximum number of", e);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testMinBlockSizeLimit() throws Exception {
+    final long blockSize = 4096;
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY, blockSize);
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
+    FileSystem fs = cluster.getFileSystem();
+
+    try {
+      // Try with min block size
+      fs.create(new Path("/testmblock1"), true, 4096, (short)3, blockSize);
+      try {
+        // Try with min block size - 1
+        fs.create(new Path("/testmblock2"), true, 4096, (short)3, blockSize-1);
+        assert false : "Expected IOException after creating a file with small" +
+            " blocks ";
+      } catch (IOException e) {
+        GenericTestUtils.assertExceptionContains("Specified block size is less",
+            e);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionFunctional.java

@@ -28,6 +28,7 @@ import java.io.IOException;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -106,10 +107,10 @@ public class TestNNStorageRetentionFunctional {
           getInProgressEditsFileName(5));
       
       LOG.info("Failing first storage dir by chmodding it");
-      sd0.setExecutable(false);
+      FileUtil.setExecutable(sd0, false);
       doSaveNamespace(nn);      
       LOG.info("Restoring accessibility of first storage dir");      
-      sd0.setExecutable(true);
+      FileUtil.setExecutable(sd0, true);
 
       LOG.info("nothing should have been purged in first storage dir");
       assertGlobEquals(cd0, "fsimage_\\d*",
@@ -138,7 +139,7 @@ public class TestNNStorageRetentionFunctional {
       assertGlobEquals(cd0, "edits_.*",
           getInProgressEditsFileName(9));
     } finally {
-      sd0.setExecutable(true);
+      FileUtil.setExecutable(sd0, true);
 
       LOG.info("Shutting down...");
       if (cluster != null) {

+ 50 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNStorageRetentionManager.java

@@ -21,6 +21,7 @@ import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getFinalizedEdits
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getImageFileName;
 import static org.apache.hadoop.hdfs.server.namenode.NNStorage.getInProgressEditsFileName;
 
+import java.io.File;
 import java.io.IOException;
 import java.util.Collection;
 import java.util.List;
@@ -247,30 +248,32 @@ public class TestNNStorageRetentionManager {
       .purgeLog(logsPurgedCaptor.capture());
 
     // Check images
-    Set<String> purgedPaths = Sets.newHashSet();
+    Set<String> purgedPaths = Sets.newLinkedHashSet();
     for (FSImageFile purged : imagesPurgedCaptor.getAllValues()) {
-      purgedPaths.add(purged.getFile().toString());
+      purgedPaths.add(fileToPath(purged.getFile()));
     }    
-    Assert.assertEquals(Joiner.on(",").join(tc.expectedPurgedImages),
-        Joiner.on(",").join(purgedPaths));
+    Assert.assertEquals(
+      Joiner.on(",").join(filesToPaths(tc.expectedPurgedImages)),
+      Joiner.on(",").join(purgedPaths));
 
     // Check images
     purgedPaths.clear();
     for (EditLogFile purged : logsPurgedCaptor.getAllValues()) {
-      purgedPaths.add(purged.getFile().toString());
+      purgedPaths.add(fileToPath(purged.getFile()));
     }    
-    Assert.assertEquals(Joiner.on(",").join(tc.expectedPurgedLogs),
-        Joiner.on(",").join(purgedPaths));
+    Assert.assertEquals(
+      Joiner.on(",").join(filesToPaths(tc.expectedPurgedLogs)),
+      Joiner.on(",").join(purgedPaths));
   }
   
   private static class TestCaseDescription {
-    private Map<String, FakeRoot> dirRoots = Maps.newHashMap();
-    private Set<String> expectedPurgedLogs = Sets.newHashSet();
-    private Set<String> expectedPurgedImages = Sets.newHashSet();
+    private Map<File, FakeRoot> dirRoots = Maps.newHashMap();
+    private Set<File> expectedPurgedLogs = Sets.newLinkedHashSet();
+    private Set<File> expectedPurgedImages = Sets.newLinkedHashSet();
     
     private static class FakeRoot {
       NameNodeDirType type;
-      List<String> files;
+      List<File> files;
       
       FakeRoot(NameNodeDirType type) {
         this.type = type;
@@ -280,33 +283,35 @@ public class TestNNStorageRetentionManager {
       StorageDirectory mockStorageDir() {
         return FSImageTestUtil.mockStorageDirectory(
             type, false,
-            files.toArray(new String[0]));
+            filesToPaths(files).toArray(new String[0]));
       }
     }
 
     void addRoot(String root, NameNodeDirType dir) {
-      dirRoots.put(root, new FakeRoot(dir));
+      dirRoots.put(new File(root), new FakeRoot(dir));
     }
 
-    private void addFile(String path) {
-      for (Map.Entry<String, FakeRoot> entry : dirRoots.entrySet()) {
-        if (path.startsWith(entry.getKey())) {
-          entry.getValue().files.add(path);
+    private void addFile(File file) {
+      for (Map.Entry<File, FakeRoot> entry : dirRoots.entrySet()) {
+        if (fileToPath(file).startsWith(fileToPath(entry.getKey()))) {
+          entry.getValue().files.add(file);
         }
       }
     }
     
     void addLog(String path, boolean expectPurge) {
-      addFile(path);
+      File file = new File(path);
+      addFile(file);
       if (expectPurge) {
-        expectedPurgedLogs.add(path);
+        expectedPurgedLogs.add(file);
       }
     }
     
     void addImage(String path, boolean expectPurge) {
-      addFile(path);
+      File file = new File(path);
+      addFile(file);
       if (expectPurge) {
-        expectedPurgedImages.add(path);
+        expectedPurgedImages.add(file);
       }
     }
     
@@ -364,6 +369,30 @@ public class TestNNStorageRetentionManager {
     }
   }
 
+  /**
+   * Converts a file to a platform-agnostic URI path.
+   * 
+   * @param file File to convert
+   * @return String path
+   */
+  private static String fileToPath(File file) {
+    return file.toURI().getPath();
+  }
+
+  /**
+   * Converts multiple files to platform-agnostic URI paths.
+   * 
+   * @param files Collection<File> files to convert
+   * @return Collection<String> paths
+   */
+  private static Collection<String> filesToPaths(Collection<File> files) {
+    List<String> paths = Lists.newArrayList();
+    for (File file: files) {
+      paths.add(fileToPath(file));
+    }
+    return paths;
+  }
+
   private static NNStorage mockStorageForDirs(final StorageDirectory ... mockDirs)
       throws IOException {
     NNStorage mockStorage = Mockito.mock(NNStorage.class);

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNNThroughputBenchmark.java

@@ -17,23 +17,35 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import java.io.File;
 import java.util.Arrays;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
 import org.junit.Test;
 
 public class TestNNThroughputBenchmark {
 
+  @After
+  public void cleanUp() {
+    FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
+  }
+
   /**
    * This test runs all benchmarks defined in {@link NNThroughputBenchmark}.
    */
   @Test
   public void testNNThroughput() throws Exception {
     Configuration conf = new HdfsConfiguration();
+    File nameDir = new File(MiniDFSCluster.getBaseDirectory(), "name");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        nameDir.getAbsolutePath());
     FileSystem.setDefaultUri(conf, "hdfs://localhost:" + 0);
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
     DFSTestUtil.formatNameNode(conf);

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameEditsConfigs.java

@@ -322,12 +322,15 @@ public class TestNameEditsConfigs {
     MiniDFSCluster cluster = null;
     File nameAndEditsDir = new File(base_dir, "name_and_edits");
     File nameAndEditsDir2 = new File(base_dir, "name_and_edits2");
+    File nameDir = new File(base_dir, "name");
 
     // 1
     // Bad configuration. Add a directory to dfs.namenode.edits.dir.required
     // without adding it to dfs.namenode.edits.dir.
     try {
       Configuration conf = new HdfsConfiguration();
+      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          nameDir.getAbsolutePath());
       conf.set(
           DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_REQUIRED_KEY,
           nameAndEditsDir2.toURI().toString());
@@ -353,6 +356,8 @@ public class TestNameEditsConfigs {
     // and dfs.namenode.edits.dir.
     try {
       Configuration conf = new HdfsConfiguration();
+      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          nameDir.getAbsolutePath());
       conf.setStrings(
           DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
           nameAndEditsDir.toURI().toString(),
@@ -375,6 +380,8 @@ public class TestNameEditsConfigs {
     // dfs.namenode.edits.dir.required.
     try {
       Configuration conf = new HdfsConfiguration();
+      conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+          nameDir.getAbsolutePath());
       conf.setStrings(
           DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
           nameAndEditsDir.toURI().toString(),

+ 17 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStorageRestore.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
+import org.apache.hadoop.util.Shell;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -362,7 +363,7 @@ public class TestStorageRestore {
       }
     }
   }
-  
+
   /**
    * 1. create DFS cluster with 3 storage directories
    *    - 2 EDITS_IMAGE(name1, name2), 1 EDITS(name3)
@@ -379,8 +380,16 @@ public class TestStorageRestore {
    */
   @Test
   public void testStorageRestoreFailure() throws Exception {
-
     SecondaryNameNode secondary = null;
+
+    // On windows, revoking write+execute permission on name2 does not
+    // prevent us from creating files in name2\current. Hence we revoke
+    // permissions on name2\current for the test.
+    String nameDir2 = Shell.WINDOWS ?
+        (new File(path2, "current").getAbsolutePath()) : path2.toString();
+    String nameDir3 = Shell.WINDOWS ?
+        (new File(path3, "current").getAbsolutePath()) : path3.toString();
+
     try {
       cluster = new MiniDFSCluster.Builder(config).numDataNodes(0)
           .manageNameDfsDirs(false).build();
@@ -394,8 +403,8 @@ public class TestStorageRestore {
       assertTrue(fs.mkdirs(path));
 
       // invalidate storage by removing rwx permission from name2 and name3
-      FileUtil.chmod(path2.toString(), "000");
-      FileUtil.chmod(path3.toString(), "000");
+      assertTrue(FileUtil.chmod(nameDir2, "000") == 0);
+      assertTrue(FileUtil.chmod(nameDir3, "000") == 0);
       secondary.doCheckpoint(); // should remove name2 and name3
 
       printStorages(cluster.getNameNode().getFSImage());
@@ -409,18 +418,18 @@ public class TestStorageRestore {
       assert (cluster.getNameNode().getFSImage().getStorage()
           .getNumStorageDirs() == 1);
 
-      FileUtil.chmod(path2.toString(), "755");
-      FileUtil.chmod(path3.toString(), "755");
+      assertTrue(FileUtil.chmod(nameDir2, "755") == 0);
+      assertTrue(FileUtil.chmod(nameDir3, "755") == 0);
       secondary.doCheckpoint(); // should restore name 2 and 3
       assert (cluster.getNameNode().getFSImage().getStorage()
           .getNumStorageDirs() == 3);
 
     } finally {
       if (path2.exists()) {
-        FileUtil.chmod(path2.toString(), "755");
+        FileUtil.chmod(nameDir2, "755");
       }
       if (path3.exists()) {
-        FileUtil.chmod(path3.toString(), "755");
+        FileUtil.chmod(nameDir3, "755");
       }
       if (cluster != null) {
         cluster.shutdown();

+ 14 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestValidateConfigurationSettings.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
 import org.junit.Test;
 
 /**
@@ -39,6 +40,11 @@ import org.junit.Test;
  */
 public class TestValidateConfigurationSettings {
 
+  @After
+  public void cleanUp() {
+    FileUtil.fullyDeleteContents(new File(MiniDFSCluster.getBaseDirectory()));
+  }
+
   /**
    * Tests setting the rpc port to the same as the web port to test that 
    * an exception
@@ -49,6 +55,10 @@ public class TestValidateConfigurationSettings {
       throws IOException {
 
     Configuration conf = new HdfsConfiguration();
+    File nameDir = new File(MiniDFSCluster.getBaseDirectory(), "name");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        nameDir.getAbsolutePath());
+
     // set both of these to port 9000, should fail
     FileSystem.setDefaultUri(conf, "hdfs://localhost:9000"); 
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
@@ -72,6 +82,10 @@ public class TestValidateConfigurationSettings {
       throws IOException {
 
     Configuration conf = new HdfsConfiguration();
+    File nameDir = new File(MiniDFSCluster.getBaseDirectory(), "name");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        nameDir.getAbsolutePath());
+
     FileSystem.setDefaultUri(conf, "hdfs://localhost:8000");
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "127.0.0.1:9000");
     DFSTestUtil.formatNameNode(conf);

+ 34 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdmin.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.test.MockitoUtil;
+import org.apache.hadoop.util.Shell;
 import org.junit.Before;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
@@ -73,6 +74,17 @@ public class TestDFSHAAdmin {
   private static String HOST_A = "1.2.3.1";
   private static String HOST_B = "1.2.3.2";
 
+  // Fencer shell commands that always return true and false respectively
+  // on Unix.
+  private static String FENCER_TRUE_COMMAND_UNIX = "shell(true)";
+  private static String FENCER_FALSE_COMMAND_UNIX = "shell(false)";
+
+  // Fencer shell commands that always return true and false respectively
+  // on Windows. Lacking POSIX 'true' and 'false' commands we use the DOS
+  // commands 'rem' and 'help.exe'.
+  private static String FENCER_TRUE_COMMAND_WINDOWS = "shell(rem)";
+  private static String FENCER_FALSE_COMMAND_WINDOWS = "shell(help.exe /? >NUL)";
+
   private HdfsConfiguration getHAConf() {
     HdfsConfiguration conf = new HdfsConfiguration();
     conf.set(DFSConfigKeys.DFS_NAMESERVICES, NSID);    
@@ -89,6 +101,16 @@ public class TestDFSHAAdmin {
     return conf;
   }
 
+  public static String getFencerTrueCommand() {
+    return Shell.WINDOWS ?
+        FENCER_TRUE_COMMAND_WINDOWS : FENCER_TRUE_COMMAND_UNIX;
+  }
+
+  public static String getFencerFalseCommand() {
+    return Shell.WINDOWS ?
+        FENCER_FALSE_COMMAND_WINDOWS : FENCER_FALSE_COMMAND_UNIX;
+  }
+
   @Before
   public void setup() throws IOException {
     mockProtocol = MockitoUtil.mockProtocol(HAServiceProtocol.class);
@@ -173,7 +195,7 @@ public class TestDFSHAAdmin {
     // Turn on auto-HA in the config
     HdfsConfiguration conf = getHAConf();
     conf.setBoolean(DFSConfigKeys.DFS_HA_AUTO_FAILOVER_ENABLED_KEY, true);
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
 
     // Should fail without the forcemanual flag
@@ -250,7 +272,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerConfigured() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2"));
   }
@@ -259,7 +281,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerAndNameservice() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-ns", "ns1", "-failover", "nn1", "nn2"));
   }
@@ -268,7 +290,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithFencerConfiguredAndForce() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
   }
@@ -277,7 +299,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithForceActive() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forceactive"));
   }
@@ -286,7 +308,7 @@ public class TestDFSHAAdmin {
   public void testFailoverWithInvalidFenceArg() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "notforcefence"));
   }
@@ -312,7 +334,7 @@ public class TestDFSHAAdmin {
     // Turn on auto-HA in the config
     HdfsConfiguration conf = getHAConf();
     conf.setBoolean(DFSConfigKeys.DFS_HA_AUTO_FAILOVER_ENABLED_KEY, true);
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
 
     assertEquals(0, runTool("-failover", "nn1", "nn2"));
@@ -323,7 +345,7 @@ public class TestDFSHAAdmin {
   public void testForceFenceOptionListedBeforeArgs() throws Exception {
     Mockito.doReturn(STANDBY_READY_RESULT).when(mockProtocol).getServiceStatus();
     HdfsConfiguration conf = getHAConf();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
   }
@@ -359,23 +381,23 @@ public class TestDFSHAAdmin {
     
     HdfsConfiguration conf = getHAConf();
     // Set the default fencer to succeed
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
     
     // Set the NN-specific fencer to fail. Should fail to fence.
-    conf.set(nnSpecificKey, "shell(false)");
+    conf.set(nnSpecificKey, getFencerFalseCommand());
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
     conf.unset(nnSpecificKey);
 
     // Set an NS-specific fencer to fail. Should fail.
-    conf.set(nsSpecificKey, "shell(false)");
+    conf.set(nsSpecificKey, getFencerFalseCommand());
     tool.setConf(conf);
     assertEquals(-1, runTool("-failover", "nn1", "nn2", "--forcefence"));
     
     // Set the NS-specific fencer to succeed. Should succeed
-    conf.set(nsSpecificKey, "shell(true)");
+    conf.set(nsSpecificKey, getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
   }

+ 21 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSHAAdminMiniCluster.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
+import org.apache.hadoop.util.Shell;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.Before;
@@ -114,7 +115,8 @@ public class TestDFSHAAdminMiniCluster {
   
   @Test
   public void testTryFailoverToSafeMode() throws Exception {
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, 
+             TestDFSHAAdmin.getFencerTrueCommand());
     tool.setConf(conf);
 
     NameNodeAdapter.enterSafeMode(cluster.getNameNode(0), false);
@@ -136,10 +138,17 @@ public class TestDFSHAAdminMiniCluster {
     // tmp file, so we can verify that the args were substituted right
     File tmpFile = File.createTempFile("testFencer", ".txt");
     tmpFile.deleteOnExit();
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
-        "shell(echo -n $target_nameserviceid.$target_namenodeid " +
-        "$target_port $dfs_ha_namenode_id > " +
-        tmpFile.getAbsolutePath() + ")");
+    if (Shell.WINDOWS) {
+      conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
+          "shell(echo %target_nameserviceid%.%target_namenodeid% " +
+              "%target_port% %dfs_ha_namenode_id% > " +
+              tmpFile.getAbsolutePath() + ")");
+    } else {
+      conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY,
+          "shell(echo -n $target_nameserviceid.$target_namenodeid " +
+          "$target_port $dfs_ha_namenode_id > " +
+          tmpFile.getAbsolutePath() + ")");
+    }
 
     // Test failover with fencer
     tool.setConf(conf);
@@ -156,9 +165,11 @@ public class TestDFSHAAdminMiniCluster {
     assertEquals(0, runTool("-failover", "nn1", "nn2", "--forcefence"));
     
     // The fence script should run with the configuration from the target
-    // node, rather than the configuration from the fencing node
-    assertEquals("minidfs-ns.nn1 " + nn1Port + " nn1",
-        Files.toString(tmpFile, Charsets.UTF_8));
+    // node, rather than the configuration from the fencing node. Strip
+    // out any trailing spaces and CR/LFs which may be present on Windows.
+    String fenceCommandOutput =Files.toString(tmpFile, Charsets.UTF_8).
+            replaceAll(" *[\r\n]+", "");
+    assertEquals("minidfs-ns.nn1 " + nn1Port + " nn1", fenceCommandOutput);
     tmpFile.delete();
     
     // Test failover with forceactive option
@@ -181,7 +192,8 @@ public class TestDFSHAAdminMiniCluster {
     assertFalse(tmpFile.exists());
 
     // Test failover with force fence listed before the other arguments
-    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, "shell(true)");
+    conf.set(DFSConfigKeys.DFS_HA_FENCE_METHODS_KEY, 
+             TestDFSHAAdmin.getFencerTrueCommand());
     tool.setConf(conf);
     assertEquals(0, runTool("-failover", "--forcefence", "nn1", "nn2"));
   }

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestGetConf.java

@@ -87,6 +87,16 @@ public class TestGetConf {
     return values;
   }
 
+  /**
+   * Add namenodes to the static resolution list to avoid going
+   * through DNS which can be really slow in some configurations.
+   */
+  private void setupStaticHostResolution(int nameServiceIdCount) {
+    for (int i = 0; i < nameServiceIdCount; i++) {
+      NetUtils.addStaticResolution("nn" + i, "localhost");
+    }
+  }
+
   /*
    * Convert the map returned from DFSUtil functions to an array of
    * addresses represented as "host:port"
@@ -306,6 +316,7 @@ public class TestGetConf {
     String[] nnAddresses = setupAddress(conf,
         DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY, nsCount, 1000);
     setupAddress(conf, DFS_NAMENODE_RPC_ADDRESS_KEY, nsCount, 1500);
+    setupStaticHostResolution(nsCount);
     String[] backupAddresses = setupAddress(conf,
         DFS_NAMENODE_BACKUP_ADDRESS_KEY, nsCount, 2000);
     String[] secondaryAddresses = setupAddress(conf,

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsUrl.java

@@ -54,7 +54,7 @@ public class TestWebHdfsUrl {
     UserGroupInformation.setConfiguration(new Configuration());
   }
   
-  @Test(timeout=4000)
+  @Test(timeout=60000)
   public void testSimpleAuthParamsInUrl() throws IOException {
     Configuration conf = new Configuration();
 
@@ -75,7 +75,7 @@ public class TestWebHdfsUrl {
         fileStatusUrl);
   }
 
-  @Test(timeout=4000)
+  @Test(timeout=60000)
   public void testSimpleProxyAuthParamsInUrl() throws IOException {
     Configuration conf = new Configuration();
 
@@ -98,7 +98,7 @@ public class TestWebHdfsUrl {
         fileStatusUrl);
   }
 
-  @Test(timeout=4000)
+  @Test(timeout=60000)
   public void testSecureAuthParamsInUrl() throws IOException {
     Configuration conf = new Configuration();
     // fake turning on security so api thinks it should use tokens
@@ -178,7 +178,7 @@ public class TestWebHdfsUrl {
         fileStatusUrl);    
   }
 
-  @Test(timeout=4000)
+  @Test(timeout=60000)
   public void testSecureProxyAuthParamsInUrl() throws IOException {
     Configuration conf = new Configuration();
     // fake turning on security so api thinks it should use tokens
@@ -290,7 +290,7 @@ public class TestWebHdfsUrl {
     return (WebHdfsFileSystem) FileSystem.get(uri, conf);
   }
   
-  @Test(timeout=4000)
+  @Test(timeout=60000)
   public void testSelectHdfsDelegationToken() throws Exception {
     SecurityUtilTestHelper.setTokenServiceUseIp(true);
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hdfs-site.xml

@@ -25,5 +25,10 @@
     <name>hadoop.security.authentication</name>
     <value>simple</value>
   </property>
+  <!-- Disable min block size since most tests use tiny blocks -->
+  <property>
+    <name>dfs.namenode.fs-limits.min-block-size</name>
+    <value>0</value>
+  </property>
 
 </configuration>

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

@@ -159,6 +159,9 @@ Trunk (Unreleased)
     HADOOP-9372. Fix bad timeout annotations on tests.
     (Arpit Agarwal via suresh)
 
+    MAPREDUCE-5177. Use common utils FileUtil#setReadable/Writable/Executable & 
+    FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
+
 Release 2.0.5-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -359,6 +362,10 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5167. Update MR App after YARN-562 to use the new builder API
     for the container. (Jian He via vinodkv)
 
+    MAPREDUCE-5179. Fix unit test in TestHSWebServices which fails when
+    versionInfo has parantheses like when running on a git checkout. (Hitesh Shah
+    via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/pipes/Application.java

@@ -113,7 +113,7 @@ class Application<K1 extends WritableComparable, V1 extends Writable,
       cmd.add(interpretor);
     }
     String executable = DistributedCache.getLocalCacheFiles(conf)[0].toString();
-    if (!new File(executable).canExecute()) {
+    if (!FileUtil.canExecute(new File(executable))) {
       // LinuxTaskController sets +x permissions on all distcache files already.
       // In case of DefaultTaskController, set permissions here.
       FileUtil.chmod(executable, "u+x");

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHsWebServices.java

@@ -348,7 +348,7 @@ public class TestHsWebServices extends JerseyTest {
       String hadoopBuildVersion, String hadoopVersion, long startedon) {
     WebServicesTestUtils.checkStringMatch("hadoopVersionBuiltOn",
         VersionInfo.getDate(), hadoopVersionBuiltOn);
-    WebServicesTestUtils.checkStringMatch("hadoopBuildVersion",
+    WebServicesTestUtils.checkStringEqual("hadoopBuildVersion",
         VersionInfo.getBuildVersion(), hadoopBuildVersion);
     WebServicesTestUtils.checkStringMatch("hadoopVersion",
         VersionInfo.getVersion(), hadoopVersion);

+ 1 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml

@@ -30,7 +30,7 @@
   <packaging>pom</packaging>
 
   <properties>
-    <!-- Needed for generating FindBugs warnings using parent pom -->
+    <hadoop.common.build.dir>${basedir}/../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
 
   <dependencies>
@@ -176,10 +176,6 @@
        <groupId>org.apache.maven.plugins</groupId>
        <artifactId>maven-surefire-plugin</artifactId>
        <configuration>
-         <environmentVariables>
-           <!-- HADOOP_HOME required for tests on Windows to find winutils -->
-           <HADOOP_HOME>${basedir}/../../../hadoop-common-project/hadoop-common/target</HADOOP_HOME>
-         </environmentVariables>
          <properties>
            <property>
              <name>listener</name>

+ 7 - 3
hadoop-project/pom.xml

@@ -50,6 +50,10 @@
 
     <test.build.dir>${project.build.directory}/test-dir</test.build.dir>
     <test.build.data>${test.build.dir}</test.build.data>
+
+    <!-- Used for building path to native library loaded by tests.  Projects -->
+    <!-- at different nesting levels in the source tree may need to override. -->
+    <hadoop.common.build.dir>${basedir}/../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
 
   <dependencyManagement>
@@ -811,8 +815,8 @@
           <argLine>-Xmx1024m -XX:+HeapDumpOnOutOfMemoryError</argLine>
           <environmentVariables>
             <!-- HADOOP_HOME required for tests on Windows to find winutils -->
-            <HADOOP_HOME>${basedir}/../../hadoop-common-project/hadoop-common/target</HADOOP_HOME>
-            <LD_LIBRARY_PATH>${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${basedir}/../../hadoop-common-project/hadoop-common/target/native/target/usr/local/lib/</LD_LIBRARY_PATH>
+            <HADOOP_HOME>${hadoop.common.build.dir}</HADOOP_HOME>
+            <LD_LIBRARY_PATH>${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib</LD_LIBRARY_PATH>
             <MALLOC_ARENA_MAX>4</MALLOC_ARENA_MAX>
           </environmentVariables>
           <systemPropertyVariables>
@@ -891,7 +895,7 @@
             <configuration>
               <environmentVariables>
                 <!-- Specify where to look for the native DLL on Windows -->
-                <PATH>${env.PATH};${basedir}/../../hadoop-common-project/hadoop-common/target/bin</PATH>
+                <PATH>${env.PATH};${hadoop.common.build.dir}/bin</PATH>
               </environmentVariables>
             </configuration>
           </plugin>

+ 2 - 1
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/PathFinder.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.streaming;
 import java.io.*;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileUtil;
 
 /**
  * Maps a relative pathname to an absolute pathname using the PATH environment.
@@ -79,7 +80,7 @@ public class PathFinder {
         f = new File(entry + fileSep + filename);
       }
       // see if the filename matches and we can read it
-      if (f.isFile() && f.canRead()) {
+      if (f.isFile() && FileUtil.canRead(f)) {
         return f;
       }
       classvalue = classvalue.substring(val + 1).trim();

+ 2 - 1
hadoop-tools/hadoop-streaming/src/main/java/org/apache/hadoop/streaming/StreamJob.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -394,7 +395,7 @@ public class StreamJob implements Tool {
   throws IllegalArgumentException {
     for (String file : values) {
       File f = new File(file);
-      if (!f.canRead()) {
+      if (!FileUtil.canRead(f)) {
         fail("File: " + f.getAbsolutePath()
           + " does not exist, or is not readable.");
       }

+ 12 - 1
hadoop-yarn-project/CHANGES.txt

@@ -41,7 +41,7 @@ Trunk - Unreleased
     classpath with new process's environment variables and localized resources
     (Chris Nauroth via bikas)
 
-  BREAKDOWN OF HADOOP-8562 SUBTASKS
+  BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
 
     YARN-158. Yarn creating package-info.java must not depend on sh.
     (Chris Nauroth via suresh)
@@ -70,6 +70,10 @@ Trunk - Unreleased
 
     YARN-359. Fixing commands for container signalling in Windows. (Chris Nauroth
     via vinodkv)
+
+    YARN-506. Move to common utils FileUtil#setReadable/Writable/Executable and 
+    FileUtil#canRead/Write/Execute. (Ivan Mitic via suresh)
+
     
 Release 2.0.5-beta - UNRELEASED
 
@@ -193,6 +197,10 @@ Release 2.0.5-beta - UNRELEASED
     YARN-591. Moved RM recovery related records out of public API as they do not
     belong there. (vinodkv)
 
+    YARN-599. Refactoring submitApplication in ClientRMService and RMAppManager
+    to separate out various validation checks depending on whether they rely on
+    RM configuration or not. (Zhijie Shen via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -312,6 +320,9 @@ Release 2.0.5-beta - UNRELEASED
     YARN-289. Fair scheduler allows reservations that won't fit on node.
     (Sandy Ryza via tomwhite)
 
+    YARN-576. Modified ResourceManager to reject NodeManagers that don't satisy
+    minimum resource requirements. (Kenji Kikushima via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 21
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/pom.xml

@@ -28,32 +28,15 @@
   <name>hadoop-yarn-applications</name>
   <packaging>pom</packaging>
 
+  <properties>
+    <hadoop.common.build.dir>${basedir}/../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
+  </properties>
+
   <modules>
     <module>hadoop-yarn-applications-distributedshell</module>
     <module>hadoop-yarn-applications-unmanaged-am-launcher</module>
   </modules>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <environmentVariables>
-            <!-- HADOOP_HOME required for tests on Windows to find winutils -->
-            <HADOOP_HOME>${basedir}/../../../../hadoop-common-project/hadoop-common/target</HADOOP_HOME>
-          </environmentVariables>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.test.TimedOutTestsListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-
  <profiles>
   <profile>
     <id>clover</id>

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

@@ -28,6 +28,7 @@ import java.util.TimerTask;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.util.StringUtils;
@@ -321,7 +322,7 @@ public class NodeHealthScriptRunner extends AbstractService {
       return false;
     }
     File f = new File(nodeHealthScript);
-    return f.exists() && f.canExecute();
+    return f.exists() && FileUtil.canExecute(f);
   }
 
   private synchronized void setHealthStatus(boolean isHealthy, String output) {

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

@@ -35,6 +35,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -307,7 +308,7 @@ public class CgroupsLCEResourcesHandler implements LCEResourcesHandler {
     if (controllerPath != null) {
       File f = new File(controllerPath + "/" + this.cgroupPrefix);
 
-      if (f.canWrite()) {
+      if (FileUtil.canWrite(f)) {
         controllerPaths.put(CONTROLLER_CPU, controllerPath);
       } else {
         throw new IOException("Not able to enforce cpu weights; cannot write "

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java

@@ -36,6 +36,7 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -104,9 +105,7 @@ public class TestLinuxContainerExecutor {
     FileContext files = FileContext.getLocalFSFileContext();
     Path workSpacePath = new Path(workSpace.getAbsolutePath());
     files.mkdir(workSpacePath, null, true);
-    workSpace.setReadable(true, false);
-    workSpace.setExecutable(true, false);
-    workSpace.setWritable(true, false);
+    FileUtil.chmod(workSpace.getAbsolutePath(), "777");
     File localDir = new File(workSpace.getAbsoluteFile(), "localDir");
     files.mkdir(new Path(localDir.getAbsolutePath()),
         new FsPermission("777"), false);

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutorWithMocks.java

@@ -37,6 +37,7 @@ import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -78,8 +79,8 @@ public class TestLinuxContainerExecutorWithMocks {
   @Before
   public void setup() {
     File f = new File("./src/test/resources/mock-container-executor");
-    if(!f.canExecute()) {
-      f.setExecutable(true);
+    if(!FileUtil.canExecute(f)) {
+      FileUtil.setExecutable(f, true);
     }
     String executorPath = f.getAbsolutePath();
     Configuration conf = new Configuration();
@@ -140,8 +141,8 @@ public class TestLinuxContainerExecutorWithMocks {
 
     // set the scheduler priority to make sure still works with nice -n prio
     File f = new File("./src/test/resources/mock-container-executor");
-    if (!f.canExecute()) {
-      f.setExecutable(true);
+    if (!FileUtil.canExecute(f)) {
+      FileUtil.setExecutable(f, true);
     }
     String executorPath = f.getAbsolutePath();
     Configuration conf = new Configuration();
@@ -204,8 +205,8 @@ public class TestLinuxContainerExecutorWithMocks {
 
     // reinitialize executer
     File f = new File("./src/test/resources/mock-container-executer-with-error");
-    if (!f.canExecute()) {
-      f.setExecutable(true);
+    if (!FileUtil.canExecute(f)) {
+      FileUtil.setExecutable(f, true);
     }
     String executorPath = f.getAbsolutePath();
     Configuration conf = new Configuration();

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeHealthService.java

@@ -28,6 +28,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -81,7 +82,7 @@ public class TestNodeHealthService {
     pw.println(scriptStr);
     pw.flush();
     pw.close();
-    nodeHealthscriptFile.setExecutable(setExecutable);
+    FileUtil.setExecutable(nodeHealthscriptFile, setExecutable);
   }
 
   @Test

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

@@ -34,6 +34,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.util.Shell;
@@ -99,7 +100,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       String timeoutCommand = Shell.WINDOWS ? "@echo \"hello\"" :
         "echo \"hello\"";
       PrintWriter writer = new PrintWriter(new FileOutputStream(shellFile));    
-      shellFile.setExecutable(true);
+      FileUtil.setExecutable(shellFile, true);
       writer.println(timeoutCommand);
       writer.close();
 
@@ -123,7 +124,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       ContainerLaunch.writeLaunchEnv(fos, env, resources, commands);
       fos.flush();
       fos.close();
-      tempFile.setExecutable(true);
+      FileUtil.setExecutable(tempFile, true);
 
       Shell.ShellCommandExecutor shexc 
       = new Shell.ShellCommandExecutor(new String[]{tempFile.getAbsolutePath()}, tmpDir);
@@ -367,7 +368,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
       writer.println("while true; do\nsleep 1s;\ndone");
     }
     writer.close();
-    scriptFile.setExecutable(true);
+    FileUtil.setExecutable(scriptFile, true);
 
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);

+ 44 - 37
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.ExitUtil;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
@@ -72,7 +71,6 @@ import org.apache.hadoop.yarn.api.records.DelegationToken;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.QueueInfo;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
@@ -83,15 +81,11 @@ import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.RMDelegationTokenSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.InvalidResourceRequestException;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNodeReport;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.authorize.RMPolicyProvider;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -266,48 +260,61 @@ public class ClientRMService extends AbstractService implements
     ApplicationSubmissionContext submissionContext = request
         .getApplicationSubmissionContext();
     ApplicationId applicationId = submissionContext.getApplicationId();
-    String user = submissionContext.getAMContainerSpec().getUser();
+
+    // ApplicationSubmissionContext needs to be validated for safety - only
+    // those fields that are independent of the RM's configuration will be
+    // checked here, those that are dependent on RM configuration are validated
+    // in RMAppManager.
+
+    String user = null;
     try {
+      // Safety
       user = UserGroupInformation.getCurrentUser().getShortUserName();
-      if (rmContext.getRMApps().get(applicationId) != null) {
-        throw new IOException("Application with id " + applicationId
-            + " is already present! Cannot add a duplicate!");
-      }
-
-      // Safety 
       submissionContext.getAMContainerSpec().setUser(user);
+    } catch (IOException ie) {
+      LOG.warn("Unable to get the current user.", ie);
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+          ie.getMessage(), "ClientRMService",
+          "Exception in submitting application", applicationId);
+      throw RPCUtil.getRemoteException(ie);
+    }
 
-      // Check whether AM resource requirements are within required limits
-      if (!submissionContext.getUnmanagedAM()) {
-        ResourceRequest amReq = BuilderUtils.newResourceRequest(
-            RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
-            submissionContext.getResource(), 1);
-        try {
-          SchedulerUtils.validateResourceRequest(amReq,
-              scheduler.getMaximumResourceCapability());
-        } catch (InvalidResourceRequestException e) {
-          LOG.warn("RM app submission failed in validating AM resource request"
-              + " for application " + applicationId, e);
-          throw RPCUtil.getRemoteException(e);
-        }
-      }
+    // Though duplication will checked again when app is put into rmContext,
+    // but it is good to fail the invalid submission as early as possible.
+    if (rmContext.getRMApps().get(applicationId) != null) {
+      String message = "Application with id " + applicationId +
+          " is already present! Cannot add a duplicate!";
+      LOG.warn(message);
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+          message, "ClientRMService", "Exception in submitting application",
+          applicationId);
+      throw RPCUtil.getRemoteException(message);
+    }
+
+    if (submissionContext.getQueue() == null) {
+      submissionContext.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
+    }
+    if (submissionContext.getApplicationName() == null) {
+      submissionContext.setApplicationName(
+          YarnConfiguration.DEFAULT_APPLICATION_NAME);
+    }
 
-      // This needs to be synchronous as the client can query 
-      // immediately following the submission to get the application status.
-      // So call handle directly and do not send an event.
-      rmAppManager.handle(new RMAppManagerSubmitEvent(submissionContext, System
-          .currentTimeMillis()));
+    try {
+      // call RMAppManager to submit application directly
+      rmAppManager.submitApplication(submissionContext,
+          System.currentTimeMillis(), false);
 
       LOG.info("Application with id " + applicationId.getId() + 
           " submitted by user " + user);
       RMAuditLogger.logSuccess(user, AuditConstants.SUBMIT_APP_REQUEST,
           "ClientRMService", applicationId);
-    } catch (IOException ie) {
-      LOG.info("Exception in submitting application", ie);
-      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST, 
-          ie.getMessage(), "ClientRMService",
+    } catch (YarnRemoteException e) {
+      LOG.info("Exception in submitting application with id " +
+          applicationId.getId(), e);
+      RMAuditLogger.logFailure(user, AuditConstants.SUBMIT_APP_REQUEST,
+          e.getMessage(), "ClientRMService",
           "Exception in submitting application", applicationId);
-      throw RPCUtil.getRemoteException(ie);
+      throw e;
     }
 
     SubmitApplicationResponse response = recordFactory

+ 63 - 52
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -31,8 +31,10 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore;
@@ -45,8 +47,12 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppRejectedEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.InvalidResourceRequestException;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 
 /**
  * This class manages the list of applications for the resource manager. 
@@ -233,64 +239,77 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
   @SuppressWarnings("unchecked")
   protected void submitApplication(
       ApplicationSubmissionContext submissionContext, long submitTime,
-      boolean isRecovered) {
+      boolean isRecovered) throws YarnRemoteException {
     ApplicationId applicationId = submissionContext.getApplicationId();
-    RMApp application = null;
-    try {
 
-      // Sanity checks
-      if (submissionContext.getQueue() == null) {
-        submissionContext.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
-      }
-      if (submissionContext.getApplicationName() == null) {
-        submissionContext.setApplicationName(
-            YarnConfiguration.DEFAULT_APPLICATION_NAME);
+    // Validation of the ApplicationSubmissionContext needs to be completed
+    // here. Only those fields that are dependent on RM's configuration are
+    // checked here as they have to be validated whether they are part of new
+    // submission or just being recovered.
+
+    // Check whether AM resource requirements are within required limits
+    if (!submissionContext.getUnmanagedAM()) {
+      ResourceRequest amReq = BuilderUtils.newResourceRequest(
+          RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
+          submissionContext.getResource(), 1);
+      try {
+        SchedulerUtils.validateResourceRequest(amReq,
+            scheduler.getMaximumResourceCapability());
+      } catch (InvalidResourceRequestException e) {
+        LOG.warn("RM app submission failed in validating AM resource request"
+            + " for application " + applicationId, e);
+        throw RPCUtil.getRemoteException(e);
       }
+    }
 
-      // Create RMApp
-      application =
-          new RMAppImpl(applicationId, rmContext, this.conf,
-              submissionContext.getApplicationName(),
-              submissionContext.getAMContainerSpec().getUser(),
-              submissionContext.getQueue(),
-              submissionContext, this.scheduler, this.masterService,
-              submitTime);
-
-      // Sanity check - duplicate?
-      if (rmContext.getRMApps().putIfAbsent(applicationId, application) != 
-          null) {
-        String message = "Application with id " + applicationId
-            + " is already present! Cannot add a duplicate!";
-        LOG.info(message);
-        throw RPCUtil.getRemoteException(message);
-      } 
+    // Create RMApp
+    RMApp application =
+        new RMAppImpl(applicationId, rmContext, this.conf,
+            submissionContext.getApplicationName(),
+            submissionContext.getAMContainerSpec().getUser(),
+            submissionContext.getQueue(),
+            submissionContext, this.scheduler, this.masterService,
+            submitTime);
 
-      // Inform the ACLs Manager
-      this.applicationACLsManager.addApplication(applicationId,
-          submissionContext.getAMContainerSpec().getApplicationACLs());
+    // Concurrent app submissions with same applicationId will fail here
+    // Concurrent app submissions with different applicationIds will not
+    // influence each other
+    if (rmContext.getRMApps().putIfAbsent(applicationId, application) !=
+        null) {
+      String message = "Application with id " + applicationId
+          + " is already present! Cannot add a duplicate!";
+      LOG.warn(message);
+      throw RPCUtil.getRemoteException(message);
+    }
+
+    // Inform the ACLs Manager
+    this.applicationACLsManager.addApplication(applicationId,
+        submissionContext.getAMContainerSpec().getApplicationACLs());
 
+    try {
       // Setup tokens for renewal
       if (UserGroupInformation.isSecurityEnabled()) {
         this.rmContext.getDelegationTokenRenewer().addApplication(
             applicationId,parseCredentials(submissionContext),
             submissionContext.getCancelTokensWhenComplete()
             );
-      }      
-      
-      // All done, start the RMApp
-      this.rmContext.getDispatcher().getEventHandler().handle(
-          new RMAppEvent(applicationId, isRecovered ? RMAppEventType.RECOVER:
-            RMAppEventType.START));
+      }
     } catch (IOException ie) {
-        LOG.info("RMAppManager submit application exception", ie);
-        if (application != null) {
-          // Sending APP_REJECTED is fine, since we assume that the 
-          // RMApp is in NEW state and thus we havne't yet informed the 
-          // Scheduler about the existence of the application
-          this.rmContext.getDispatcher().getEventHandler().handle(
-              new RMAppRejectedEvent(applicationId, ie.getMessage()));
-        }
+      LOG.warn(
+          "Unable to add the application to the delegation token renewer.",
+          ie);
+      // Sending APP_REJECTED is fine, since we assume that the
+      // RMApp is in NEW state and thus we havne't yet informed the
+      // Scheduler about the existence of the application
+      this.rmContext.getDispatcher().getEventHandler().handle(
+          new RMAppRejectedEvent(applicationId, ie.getMessage()));
+      throw RPCUtil.getRemoteException(ie);
     }
+
+    // All done, start the RMApp
+    this.rmContext.getDispatcher().getEventHandler().handle(
+        new RMAppEvent(applicationId, isRecovered ? RMAppEventType.RECOVER:
+            RMAppEventType.START));
   }
   
   private Credentials parseCredentials(ApplicationSubmissionContext application) 
@@ -377,14 +396,6 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
         checkAppNumCompletedLimit(); 
       } 
       break;
-      case APP_SUBMIT:
-      {
-        ApplicationSubmissionContext submissionContext = 
-            ((RMAppManagerSubmitEvent)event).getSubmissionContext();
-        long submitTime = ((RMAppManagerSubmitEvent)event).getSubmitTime();
-        submitApplication(submissionContext, submitTime, false);
-      }
-      break;
       default:
         LOG.error("Invalid eventtype " + event.getType() + ". Ignoring!");
       }

+ 0 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerEventType.java

@@ -19,6 +19,5 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 public enum RMAppManagerEventType {
-  APP_SUBMIT,
   APP_COMPLETED
 }

+ 0 - 43
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerSubmitEvent.java

@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.resourcemanager;
-
-import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-
-public class RMAppManagerSubmitEvent extends RMAppManagerEvent {
-
-  private final ApplicationSubmissionContext submissionContext;
-  private final long submitTime;
-
-  public RMAppManagerSubmitEvent(
-      ApplicationSubmissionContext submissionContext, long submitTime) {
-    super(submissionContext.getApplicationId(),
-        RMAppManagerEventType.APP_SUBMIT);
-    this.submissionContext = submissionContext;
-    this.submitTime = submitTime;
-  }
-
-  public ApplicationSubmissionContext getSubmissionContext() {
-    return this.submissionContext;
-  }
-  
-  public long getSubmitTime() {
-    return this.submitTime;
-  }
-}

+ 21 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -78,6 +78,9 @@ public class ResourceTrackerService extends AbstractService implements
   private static final NodeHeartbeatResponse shutDown = recordFactory
   .newRecordInstance(NodeHeartbeatResponse.class);
   
+  private int minAllocMb;
+  private int minAllocVcores;
+
   static {
     resync.setNodeAction(NodeAction.RESYNC);
 
@@ -111,6 +114,14 @@ public class ResourceTrackerService extends AbstractService implements
           + YarnConfiguration.RM_NM_HEARTBEAT_INTERVAL_MS
           + " should be larger than 0.");
     }
+
+    minAllocMb = conf.getInt(
+    	YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB,
+    	YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
+    minAllocVcores = conf.getInt(
+    	YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
+    	YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
+    
     super.init(conf);
   }
 
@@ -169,6 +180,16 @@ public class ResourceTrackerService extends AbstractService implements
       return response;
     }
 
+    // Check if this node has minimum allocations
+    if (capability.getMemory() < minAllocMb
+        || capability.getVirtualCores() < minAllocVcores) {
+      LOG.info("NodeManager from  " + host
+          + " doesn't satisfy minimum allocations, Sending SHUTDOWN"
+          + " signal to the NodeManager.");
+      response.setNodeAction(NodeAction.SHUTDOWN);
+      return response;
+    }
+
     if (isSecurityEnabled()) {
       MasterKey nextMasterKeyForNode =
           this.containerTokenSecretManager.getCurrentKey();

+ 3 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java

@@ -45,6 +45,7 @@ public class MockNM {
   private int responseId;
   private NodeId nodeId;
   private final int memory;
+  private final int vCores = 1;
   private ResourceTrackerService resourceTracker;
   private final int httpPort = 2;
   private MasterKey currentMasterKey;
@@ -53,9 +54,7 @@ public class MockNM {
     this.memory = memory;
     this.resourceTracker = resourceTracker;
     String[] splits = nodeIdStr.split(":");
-    nodeId = Records.newRecord(NodeId.class);
-    nodeId.setHost(splits[0]);
-    nodeId.setPort(Integer.parseInt(splits[1]));
+    nodeId = BuilderUtils.newNodeId(splits[0], Integer.parseInt(splits[1]));
   }
 
   public NodeId getNodeId() {
@@ -83,8 +82,7 @@ public class MockNM {
         RegisterNodeManagerRequest.class);
     req.setNodeId(nodeId);
     req.setHttpPort(httpPort);
-    Resource resource = Records.newRecord(Resource.class);
-    resource.setMemory(memory);
+    Resource resource = BuilderUtils.newResource(memory, vCores);
     req.setResource(resource);
     RegisterNodeManagerResponse registrationResponse =
         resourceTracker.registerNodeManager(req);

+ 110 - 133
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -19,6 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 import java.util.HashMap;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
@@ -31,12 +34,15 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
@@ -46,11 +52,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessM
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.ContainerAllocationExpirer;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.service.Service;
-import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.collect.Lists;
@@ -163,9 +169,10 @@ public class TestAppManager{
       super.setCompletedAppsMax(max);
     }
     public void submitApplication(
-        ApplicationSubmissionContext submissionContext) {
-      super.submitApplication(
-          submissionContext, System.currentTimeMillis(), false);
+        ApplicationSubmissionContext submissionContext)
+            throws YarnRemoteException {
+      super.submitApplication(submissionContext, System.currentTimeMillis(),
+          false);
     }
   }
 
@@ -179,6 +186,40 @@ public class TestAppManager{
     }
   }
 
+  private RMContext rmContext;
+  private TestRMAppManager appMonitor;
+  private ApplicationSubmissionContext asContext;
+  private ApplicationId appId;
+
+  @Before
+  public void setUp() {
+    long now = System.currentTimeMillis();
+
+    rmContext = mockRMContext(1, now - 10);
+    ResourceScheduler scheduler = mockResourceScheduler();
+    Configuration conf = new Configuration();
+    ApplicationMasterService masterService =
+        new ApplicationMasterService(rmContext, scheduler);
+    appMonitor = new TestRMAppManager(rmContext,
+        new ClientToAMTokenSecretManagerInRM(), scheduler, masterService,
+        new ApplicationACLsManager(conf), conf);
+
+    appId = MockApps.newAppID(1);
+    RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+    asContext =
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    asContext.setApplicationId(appId);
+    asContext.setAMContainerSpec(mockContainerLaunchContext(recordFactory));
+    asContext.setResource(mockResource());
+    setupDispatcher(rmContext, conf);
+  }
+
+  @After
+  public void tearDown() {
+    setAppEventType(RMAppEventType.KILL);
+    ((Service)rmContext.getDispatcher()).stop();
+  }
+
   @Test
   public void testRMAppRetireNone() throws Exception {
     long now = System.currentTimeMillis();
@@ -334,38 +375,10 @@ public class TestAppManager{
 
   @Test
   public void testRMAppSubmit() throws Exception {
-    long now = System.currentTimeMillis();
-
-    RMContext rmContext = mockRMContext(0, now - 10);
-    ResourceScheduler scheduler = new CapacityScheduler();
-    Configuration conf = new Configuration();
-    ApplicationMasterService masterService =
-        new ApplicationMasterService(rmContext, scheduler);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext,
-        new ClientToAMTokenSecretManagerInRM(), scheduler, masterService,
-        new ApplicationACLsManager(conf), conf);
-
-    ApplicationId appID = MockApps.newAppID(1);
-    RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    ApplicationSubmissionContext context = 
-        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
-    context.setApplicationId(appID);
-    ContainerLaunchContext amContainer = recordFactory
-        .newRecordInstance(ContainerLaunchContext.class);
-    amContainer.setApplicationACLs(new HashMap<ApplicationAccessType, String>());
-    context.setAMContainerSpec(amContainer);
-    setupDispatcher(rmContext, conf);
-
-    appMonitor.submitApplication(context);
-    RMApp app = rmContext.getRMApps().get(appID);
+    appMonitor.submitApplication(asContext);
+    RMApp app = rmContext.getRMApps().get(appId);
     Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
-    Assert.assertEquals("app name doesn't match", 
-        YarnConfiguration.DEFAULT_APPLICATION_NAME, 
-        app.getName());
-    Assert.assertEquals("app queue doesn't match", 
-        YarnConfiguration.DEFAULT_QUEUE_NAME, 
-        app.getQueue());
+    Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
     Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
 
     // wait for event to be processed
@@ -374,9 +387,8 @@ public class TestAppManager{
         timeoutSecs++ < 20) {
       Thread.sleep(1000);
     }
-    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START, getAppEventType());
-    setAppEventType(RMAppEventType.KILL); 
-    ((Service)rmContext.getDispatcher()).stop();
+    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START,
+        getAppEventType());
   }
 
   @Test (timeout = 30000)
@@ -390,10 +402,7 @@ public class TestAppManager{
         new int[]{ 1, 1, 1, 1 }};
     for (int i = 0; i < globalMaxAppAttempts.length; ++i) {
       for (int j = 0; j < individualMaxAppAttempts.length; ++j) {
-        long now = System.currentTimeMillis();
-
-        RMContext rmContext = mockRMContext(0, now - 10);
-        ResourceScheduler scheduler = new CapacityScheduler();
+        ResourceScheduler scheduler = mockResourceScheduler();
         Configuration conf = new Configuration();
         conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, globalMaxAppAttempts[i]);
         ApplicationMasterService masterService =
@@ -402,21 +411,12 @@ public class TestAppManager{
             new ClientToAMTokenSecretManagerInRM(), scheduler, masterService,
             new ApplicationACLsManager(conf), conf);
 
-        RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-        ApplicationSubmissionContext context =
-            recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
-        ContainerLaunchContext amContainer = recordFactory
-            .newRecordInstance(ContainerLaunchContext.class);
-        amContainer.setApplicationACLs(new HashMap<ApplicationAccessType, String>());
-        context.setAMContainerSpec(amContainer);
-        setupDispatcher(rmContext, conf);
-
-        ApplicationId appID = MockApps.newAppID(1);
-        context.setApplicationId(appID);
+        ApplicationId appID = MockApps.newAppID(i * 4 + j + 1);
+        asContext.setApplicationId(appID);
         if (individualMaxAppAttempts[i][j] != 0) {
-          context.setMaxAppAttempts(individualMaxAppAttempts[i][j]);
+          asContext.setMaxAppAttempts(individualMaxAppAttempts[i][j]);
         }
-        appMonitor.submitApplication(context);
+        appMonitor.submitApplication(asContext);
         RMApp app = rmContext.getRMApps().get(appID);
         Assert.assertEquals("max application attempts doesn't match",
             expectedNums[i][j], app.getMaxAppAttempts());
@@ -428,96 +428,73 @@ public class TestAppManager{
           Thread.sleep(1000);
         }
         setAppEventType(RMAppEventType.KILL);
-        ((Service)rmContext.getDispatcher()).stop();
       }
     }
   }
 
-  @Test (timeout = 3000)
-  public void testRMAppSubmitWithQueueAndName() throws Exception {
-    long now = System.currentTimeMillis();
-
-    RMContext rmContext = mockRMContext(1, now - 10);
-    ResourceScheduler scheduler = new CapacityScheduler();
-    Configuration conf = new Configuration();
-    ApplicationMasterService masterService =
-        new ApplicationMasterService(rmContext, scheduler);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext,
-        new ClientToAMTokenSecretManagerInRM(), scheduler, masterService,
-        new ApplicationACLsManager(conf), conf);
-
-    ApplicationId appID = MockApps.newAppID(10);
-    RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
-    context.setApplicationId(appID);
-    context.setApplicationName("testApp1");
-    context.setQueue("testQueue");
-    ContainerLaunchContext amContainer = recordFactory
-        .newRecordInstance(ContainerLaunchContext.class);
-    amContainer
-        .setApplicationACLs(new HashMap<ApplicationAccessType, String>());
-    context.setAMContainerSpec(amContainer);
+  @Test (timeout = 30000)
+  public void testRMAppSubmitDuplicateApplicationId() throws Exception {
+    ApplicationId appId = MockApps.newAppID(0);
+    asContext.setApplicationId(appId);
+    RMApp appOrig = rmContext.getRMApps().get(appId);
+    Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
 
-    setupDispatcher(rmContext, conf);
+    // our testApp1 should be rejected and original app with same id should be left in place
+    try {
+      appMonitor.submitApplication(asContext);
+      Assert.fail("Exception is expected when applicationId is duplicate.");
+    } catch (YarnRemoteException e) {
+      Assert.assertTrue("The thrown exception is not the expectd one.",
+          e.getMessage().contains("Cannot add a duplicate!"));
+    }
 
-    appMonitor.submitApplication(context);
-    RMApp app = rmContext.getRMApps().get(appID);
+    // make sure original app didn't get removed
+    RMApp app = rmContext.getRMApps().get(appId);
     Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
-    Assert.assertEquals("app name doesn't match", "testApp1", app.getName());
-    Assert.assertEquals("app queue doesn't match", "testQueue", app.getQueue());
-    Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
+    Assert.assertEquals("app id doesn't match", appId, app.getApplicationId());
+    Assert.assertEquals("app state doesn't match", RMAppState.FINISHED, app.getState());
+  }
 
-    // wait for event to be processed
-    int timeoutSecs = 0;
-    while ((getAppEventType() == RMAppEventType.KILL) && 
-        timeoutSecs++ < 20) {
-      Thread.sleep(1000);
+  @Test (timeout = 30000)
+  public void testRMAppSubmitInvalidResourceRequest() throws Exception {
+    asContext.setResource(Resources.createResource(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1));
+
+    // submit an app
+    try {
+      appMonitor.submitApplication(asContext);
+      Assert.fail("Application submission should fail because resource" +
+          " request is invalid.");
+    } catch (YarnRemoteException e) {
+      // Exception is expected
+      Assert.assertTrue("The thrown exception is not" +
+          " InvalidResourceRequestException",
+          e.getMessage().startsWith("Invalid resource request"));
     }
-    Assert.assertEquals("app event type sent is wrong", RMAppEventType.START, getAppEventType());
-    setAppEventType(RMAppEventType.KILL); 
-    ((Service)rmContext.getDispatcher()).stop();
   }
 
-  @Test
-  public void testRMAppSubmitError() throws Exception {
-    long now = System.currentTimeMillis();
-
-    // specify 1 here and use same appId below so it gets duplicate entry
-    RMContext rmContext = mockRMContext(1, now - 10);
-    ResourceScheduler scheduler = new CapacityScheduler();
-    Configuration conf = new Configuration();
-    ApplicationMasterService masterService =
-        new ApplicationMasterService(rmContext, scheduler);
-    TestRMAppManager appMonitor = new TestRMAppManager(rmContext,
-        new ClientToAMTokenSecretManagerInRM(), scheduler, masterService,
-        new ApplicationACLsManager(conf), conf);
-
-    ApplicationId appID = MockApps.newAppID(0);
-    RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
-    context.setApplicationId(appID);
-    context.setApplicationName("testApp1");
-    context.setQueue("testQueue");
-
-    setupDispatcher(rmContext, conf);
-
-    RMApp appOrig = rmContext.getRMApps().get(appID);
-    Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
+  private static ResourceScheduler mockResourceScheduler() {
+    ResourceScheduler scheduler = mock(ResourceScheduler.class);
+    when(scheduler.getMinimumResourceCapability()).thenReturn(
+        Resources.createResource(
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
+    when(scheduler.getMaximumResourceCapability()).thenReturn(
+        Resources.createResource(
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
+    return scheduler;
+  }
 
-    ContainerLaunchContext clc =
-        BuilderUtils.newContainerLaunchContext(null, null, null, null, null,
-            null, null);
-    context.setAMContainerSpec(clc);
-    // our testApp1 should be rejected and original app with same id should be left in place
-    appMonitor.submitApplication(context);
+  private static ContainerLaunchContext mockContainerLaunchContext(
+      RecordFactory recordFactory) {
+    ContainerLaunchContext amContainer = recordFactory.newRecordInstance(
+        ContainerLaunchContext.class);
+    amContainer.setApplicationACLs(new HashMap<ApplicationAccessType, String>());;
+    return amContainer;
+  }
 
-    // make sure original app didn't get removed
-    RMApp app = rmContext.getRMApps().get(appID);
-    Assert.assertNotNull("app is null", app);
-    Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
-    Assert.assertEquals("app name doesn't matches", appOrig.getName(), app.getName());
-    ((Service)rmContext.getDispatcher()).stop();
+  private static Resource mockResource() {
+    return Resources.createResource(
+        YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
   }
 
 }

+ 85 - 57
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -250,17 +250,70 @@ public class TestClientRMService {
         rmContext, null, null, null, dtsm);
     rmService.renewDelegationToken(request);
   }
+
+  @Test (timeout = 30000)
+  @SuppressWarnings ("rawtypes")
+  public void testAppSubmit() throws Exception {
+    YarnScheduler yarnScheduler = mockYarnScheduler();
+    RMContext rmContext = mock(RMContext.class);
+    mockRMContext(yarnScheduler, rmContext);
+    RMStateStore stateStore = mock(RMStateStore.class);
+    when(rmContext.getStateStore()).thenReturn(stateStore);
+    RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
+        null, mock(ApplicationACLsManager.class), new Configuration());
+    when(rmContext.getDispatcher().getEventHandler()).thenReturn(
+        new EventHandler<Event>() {
+          public void handle(Event event) {}
+        });
+    ClientRMService rmService =
+        new ClientRMService(rmContext, yarnScheduler, appManager, null, null);
+
+    // without name and queue
+    ApplicationId appId1 = getApplicationId(100);
+    SubmitApplicationRequest submitRequest1 = mockSubmitAppRequest(
+        appId1, null, null);
+    try {
+      rmService.submitApplication(submitRequest1);
+    } catch (YarnRemoteException e) {
+      Assert.fail("Exception is not expected.");
+    }
+    RMApp app1 = rmContext.getRMApps().get(appId1);
+    Assert.assertNotNull("app doesn't exist", app1);
+    Assert.assertEquals("app name doesn't match",
+        YarnConfiguration.DEFAULT_APPLICATION_NAME, app1.getName());
+    Assert.assertEquals("app queue doesn't match",
+        YarnConfiguration.DEFAULT_QUEUE_NAME, app1.getQueue());
+
+    // with name and queue
+    String name = MockApps.newAppName();
+    String queue = MockApps.newQueue();
+    ApplicationId appId2 = getApplicationId(101);
+    SubmitApplicationRequest submitRequest2 = mockSubmitAppRequest(
+        appId2, name, queue);
+    try {
+      rmService.submitApplication(submitRequest2);
+    } catch (YarnRemoteException e) {
+      Assert.fail("Exception is not expected.");
+    }
+    RMApp app2 = rmContext.getRMApps().get(appId2);
+    Assert.assertNotNull("app doesn't exist", app2);
+    Assert.assertEquals("app name doesn't match", name, app2.getName());
+    Assert.assertEquals("app queue doesn't match", queue, app2.getQueue());
+
+    // duplicate appId
+    try {
+      rmService.submitApplication(submitRequest2);
+      Assert.fail("Exception is expected.");
+    } catch (YarnRemoteException e) {
+      Assert.assertTrue("The thrown exception is not expected.",
+          e.getMessage().contains("Cannot add a duplicate!"));
+    }
+  }
   
   @Test(timeout=4000)
   public void testConcurrentAppSubmit()
       throws IOException, InterruptedException, BrokenBarrierException {
-    YarnScheduler yarnScheduler = mock(YarnScheduler.class);
-    when(yarnScheduler.getMinimumResourceCapability()).thenReturn(
-        Resources.createResource(
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
-    when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
-        Resources.createResource(
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
+    YarnScheduler yarnScheduler = mockYarnScheduler();
     RMContext rmContext = mock(RMContext.class);
     mockRMContext(yarnScheduler, rmContext);
     RMStateStore stateStore = mock(RMStateStore.class);
@@ -270,8 +323,10 @@ public class TestClientRMService {
 
     final ApplicationId appId1 = getApplicationId(100);
     final ApplicationId appId2 = getApplicationId(101);
-    final SubmitApplicationRequest submitRequest1 = mockSubmitAppRequest(appId1);
-    final SubmitApplicationRequest submitRequest2 = mockSubmitAppRequest(appId2);
+    final SubmitApplicationRequest submitRequest1 = mockSubmitAppRequest(
+        appId1, null, null);
+    final SubmitApplicationRequest submitRequest2 = mockSubmitAppRequest(
+        appId2, null, null);
     
     final CyclicBarrier startBarrier = new CyclicBarrier(2);
     final CyclicBarrier endBarrier = new CyclicBarrier(2);
@@ -319,61 +374,23 @@ public class TestClientRMService {
     t.join();
   }
 
-  @Test (timeout = 30000)
-  public void testInvalidResourceRequestWhenSubmittingApplication()
-      throws IOException, InterruptedException, BrokenBarrierException {
-    YarnScheduler yarnScheduler = mock(YarnScheduler.class);
-    when(yarnScheduler.getMinimumResourceCapability()).thenReturn(
-        Resources.createResource(
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
-    when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
-        Resources.createResource(
-            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
-    RMContext rmContext = mock(RMContext.class);
-    mockRMContext(yarnScheduler, rmContext);
-    RMStateStore stateStore = mock(RMStateStore.class);
-    when(rmContext.getStateStore()).thenReturn(stateStore);
-    RMAppManager appManager = new RMAppManager(rmContext, yarnScheduler,
-        null, mock(ApplicationACLsManager.class), new Configuration());
-
-    final ApplicationId appId = getApplicationId(100);
-    final SubmitApplicationRequest submitRequest = mockSubmitAppRequest(appId);
-    Resource resource = Resources.createResource(
-        YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1);
-    when(submitRequest.getApplicationSubmissionContext()
-        .getResource()).thenReturn(resource);
-
-    final ClientRMService rmService =
-        new ClientRMService(rmContext, yarnScheduler, appManager, null, null);
-
-    // submit an app
-    try {
-      rmService.submitApplication(submitRequest);
-      Assert.fail("Application submission should fail because resource" +
-          " request is invalid.");
-    } catch (YarnRemoteException e) {
-      // Exception is expected
-      Assert.assertTrue("The thrown exception is not" +
-          " InvalidResourceRequestException",
-          e.getMessage().startsWith("Invalid resource request"));
-    }
-  }
- 
-  private SubmitApplicationRequest mockSubmitAppRequest(ApplicationId appId) {
+  private SubmitApplicationRequest mockSubmitAppRequest(ApplicationId appId,
+      String name, String queue) {
     String user = MockApps.newUserName();
-    String queue = MockApps.newQueue();
 
     ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
 
     Resource resource = Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
 
-    ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
-    when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
-    when(submissionContext.getAMContainerSpec().getUser()).thenReturn(user);
-    when(submissionContext.getQueue()).thenReturn(queue);
-    when(submissionContext.getApplicationId()).thenReturn(appId);
-    when(submissionContext.getResource()).thenReturn(resource);
+    ApplicationSubmissionContext submissionContext =
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    submissionContext.setAMContainerSpec(amContainerSpec);
+    submissionContext.getAMContainerSpec().setUser(user);
+    submissionContext.setApplicationName(name);
+    submissionContext.setQueue(queue);
+    submissionContext.setApplicationId(appId);
+    submissionContext.setResource(resource);
 
    SubmitApplicationRequest submitRequest =
        recordFactory.newRecordInstance(SubmitApplicationRequest.class);
@@ -429,4 +446,15 @@ public class TestClientRMService {
         queueName, asContext, yarnScheduler, null , System
             .currentTimeMillis());
   }
+
+  private static YarnScheduler mockYarnScheduler() {
+    YarnScheduler yarnScheduler = mock(YarnScheduler.class);
+    when(yarnScheduler.getMinimumResourceCapability()).thenReturn(
+        Resources.createResource(
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB));
+    when(yarnScheduler.getMaximumResourceCapability()).thenReturn(
+        Resources.createResource(
+            YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB));
+    return yarnScheduler;
+  }
 }

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -43,6 +44,7 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResp
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.Test;
@@ -283,6 +285,49 @@ public class TestResourceTrackerService {
       response.getRMIdentifier());
   }
 
+  @Test
+  public void testNodeRegistrationWithMinimumAllocations() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, "2048");
+    conf.set(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, "4");
+    rm = new MockRM(conf);
+    rm.start();
+
+    ResourceTrackerService resourceTrackerService
+      = rm.getResourceTrackerService();
+    RegisterNodeManagerRequest req = Records.newRecord(
+        RegisterNodeManagerRequest.class);
+    NodeId nodeId = BuilderUtils.newNodeId("host", 1234);
+    req.setNodeId(nodeId);
+
+    Resource capability = BuilderUtils.newResource(1024, 1);
+    req.setResource(capability);
+    RegisterNodeManagerResponse response1 =
+        resourceTrackerService.registerNodeManager(req);
+    Assert.assertEquals(NodeAction.SHUTDOWN,response1.getNodeAction());
+    
+    capability.setMemory(2048);
+    capability.setVirtualCores(1);
+    req.setResource(capability);
+    RegisterNodeManagerResponse response2 =
+        resourceTrackerService.registerNodeManager(req);
+    Assert.assertEquals(NodeAction.SHUTDOWN,response2.getNodeAction());
+    
+    capability.setMemory(1024);
+    capability.setVirtualCores(4);
+    req.setResource(capability);
+    RegisterNodeManagerResponse response3 =
+        resourceTrackerService.registerNodeManager(req);
+    Assert.assertEquals(NodeAction.SHUTDOWN,response3.getNodeAction());
+    
+    capability.setMemory(2048);
+    capability.setVirtualCores(4);
+    req.setResource(capability);
+    RegisterNodeManagerResponse response4 =
+        resourceTrackerService.registerNodeManager(req);
+    Assert.assertEquals(NodeAction.NORMAL,response4.getNodeAction());
+  }
+
   @Test
   public void testReboot() throws Exception {
     Configuration conf = new Configuration();

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Before;
 import org.junit.Test;
@@ -127,7 +128,7 @@ public class TestNMExpiry {
     String hostname1 = "localhost1";
     String hostname2 = "localhost2";
     String hostname3 = "localhost3";
-    Resource capability = recordFactory.newRecordInstance(Resource.class);
+    Resource capability = BuilderUtils.newResource(1024, 1);
 
     RegisterNodeManagerRequest request1 = recordFactory
         .newRecordInstance(RegisterNodeManagerRequest.class);

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestRMNMRPCResponseId.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.ResourceTrackerService;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.Assert;
@@ -89,7 +90,7 @@ public class TestRMNMRPCResponseId {
   @Test
   public void testRPCResponseId() throws IOException {
     String node = "localhost";
-    Resource capability = recordFactory.newRecordInstance(Resource.class);
+    Resource capability = BuilderUtils.newResource(1024, 1);
     RegisterNodeManagerRequest request = recordFactory.newRecordInstance(RegisterNodeManagerRequest.class);
     nodeId = Records.newRecord(NodeId.class);
     nodeId.setHost(node);

+ 3 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/pom.xml

@@ -28,26 +28,9 @@
   <name>hadoop-yarn-server</name>
   <packaging>pom</packaging>
 
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <environmentVariables>
-            <!-- HADOOP_HOME required for tests on Windows to find winutils -->
-            <HADOOP_HOME>${basedir}/../../../../hadoop-common-project/hadoop-common/target</HADOOP_HOME>
-          </environmentVariables>
-          <properties>
-            <property>
-              <name>listener</name>
-              <value>org.apache.hadoop.test.TimedOutTestsListener</value>
-            </property>
-          </properties>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
+  <properties>
+    <hadoop.common.build.dir>${basedir}/../../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
+  </properties>
 
   <dependencies>
     <dependency>

+ 1 - 4
hadoop-yarn-project/hadoop-yarn/pom.xml

@@ -30,6 +30,7 @@
     <test.logs>true</test.logs>
     <test.timeout>600000</test.timeout>
     <yarn.basedir>${basedir}</yarn.basedir>
+    <hadoop.common.build.dir>${basedir}/../../../hadoop-common-project/hadoop-common/target</hadoop.common.build.dir>
   </properties>
 
   <dependencies>
@@ -159,10 +160,6 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
-          <environmentVariables>
-            <!-- HADOOP_HOME required for tests on Windows to find winutils -->
-            <HADOOP_HOME>${basedir}/../../../hadoop-common-project/hadoop-common/target</HADOOP_HOME>
-          </environmentVariables>
           <properties>
             <property>
               <name>listener</name>