瀏覽代碼

Merge trunk to branch-trunk-win

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-trunk-win@1451695 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 年之前
父節點
當前提交
9755a8d4ed
共有 99 個文件被更改,包括 2234 次插入810 次删除
  1. 27 0
      BUILDING.txt
  2. 2 2
      dev-support/test-patch.sh
  3. 34 1
      hadoop-common-project/hadoop-common/CHANGES.txt
  4. 4 0
      hadoop-common-project/hadoop-common/pom.xml
  5. 2 1
      hadoop-common-project/hadoop-common/src/CMakeLists.txt
  6. 6 0
      hadoop-common-project/hadoop-common/src/main/bin/hadoop
  7. 1 19
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java
  8. 78 33
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java
  9. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  10. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  11. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/PositionedReadable.java
  12. 5 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java
  13. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java
  14. 2 2
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java
  15. 8 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  16. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/record/Buffer.java
  17. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/record/RecordOutput.java
  18. 2 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java
  19. 60 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ClassUtil.java
  20. 7 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java
  21. 79 3
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFVariations.java
  22. 29 30
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java
  23. 40 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClassUtil.java
  24. 18 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  25. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt
  26. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java
  27. 8 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
  28. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
  29. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java
  30. 12 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  31. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java
  32. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  33. 11 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  34. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  35. 129 107
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  36. 37 19
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
  37. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
  38. 7 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java
  39. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java
  40. 137 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java
  41. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
  42. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
  43. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  44. 52 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java
  45. 19 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
  46. 70 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java
  47. 27 0
      hadoop-mapreduce-project/CHANGES.txt
  48. 8 0
      hadoop-mapreduce-project/bin/mapred
  49. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  50. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java
  51. 18 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
  52. 27 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  53. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr
  54. 4 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java
  55. 17 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java
  56. 53 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java
  57. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/QueueConfigurationParser.java
  58. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  59. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  60. 123 44
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
  61. 75 24
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java
  62. 136 80
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java
  63. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReader.java
  64. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java
  65. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java
  66. 4 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryParsing.java
  67. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  68. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
  69. 86 33
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  70. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  71. 2 2
      hadoop-mapreduce-project/pom.xml
  72. 2 2
      hadoop-project/pom.xml
  73. 0 71
      hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestUniformSizeInputFormat.java
  74. 4 4
      hadoop-tools/hadoop-pipes/src/CMakeLists.txt
  75. 11 3
      hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java
  76. 18 0
      hadoop-yarn-project/CHANGES.txt
  77. 19 7
      hadoop-yarn-project/hadoop-yarn/bin/yarn
  78. 34 27
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java
  79. 35 28
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java
  80. 58 34
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  81. 17 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestRackResolver.java
  82. 11 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  83. 113 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
  84. 0 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  85. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  86. 16 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java
  87. 58 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java
  88. 45 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/UpdatedContainerInfo.java
  89. 12 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java
  90. 1 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java
  91. 11 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
  92. 10 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  93. 12 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java
  94. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  95. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestFifoScheduler.java
  96. 150 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java
  97. 29 48
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  98. 1 1
      hadoop-yarn-project/hadoop-yarn/pom.xml
  99. 2 2
      hadoop-yarn-project/pom.xml

+ 27 - 0
BUILDING.txt

@@ -87,6 +87,33 @@ Maven build goals:
   * -Dtest.exclude=<TESTCLASSNAME>
   * -Dtest.exclude.pattern=**/<TESTCLASSNAME1>.java,**/<TESTCLASSNAME2>.java
 
+----------------------------------------------------------------------------------
+Building components separately
+
+If you are building a submodule directory, all the hadoop dependencies this
+submodule has will be resolved as all other 3rd party dependencies. This is,
+from the Maven cache or from a Maven repository (if not available in the cache
+or the SNAPSHOT 'timed out').
+An alternative is to run 'mvn install -DskipTests' from Hadoop source top
+level once; and then work from the submodule. Keep in mind that SNAPSHOTs
+time out after a while, using the Maven '-nsu' will stop Maven from trying
+to update SNAPSHOTs from external repos.
+
+----------------------------------------------------------------------------------
+Importing projects to eclipse
+
+When you import the project to eclipse, install hadoop-maven-plugins at first.
+
+  $ cd hadoop-maven-plugins
+  $ mvn install
+
+Then, generate ecplise project files.
+
+  $ mvn eclipse:eclipse -DskipTests
+
+At last, import to eclipse by specifying the root directory of the project via
+[File] > [Import] > [Existing Projects into Workspace].
+
 ----------------------------------------------------------------------------------
 Building distributions:
 

+ 2 - 2
dev-support/test-patch.sh

@@ -370,12 +370,12 @@ checkTests () {
     JIRA_COMMENT="$JIRA_COMMENT
 
     {color:green}+1 tests included appear to have a timeout.{color}"
-	return 1
+	return 0
   fi
   JIRA_COMMENT="$JIRA_COMMENT
 
   {color:red}-1 one of tests included doesn't have a timeout.{color}"
-  return 0
+  return 1
 }
 
 cleanUpXml () {

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

@@ -355,6 +355,11 @@ Release 2.0.4-beta - UNRELEASED
 
     HADOOP-9117. replace protoc ant plugin exec with a maven plugin. (tucu)
 
+    HADOOP-9279. Document the need to build hadoop-maven-plugins for
+    eclipse and separate project builds. (Tsuyoshi Ozawa via suresh)
+
+    HADOOP-9334. Upgrade netty version. (Nicolas Liochon via suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -376,6 +381,28 @@ Release 2.0.4-beta - UNRELEASED
 
     HADOOP-9304. remove addition of avro genreated-sources dirs to build. (tucu)
 
+    HADOOP-9267. hadoop -help, -h, --help should show usage instructions.
+    (Andrew Wang via atm)
+
+    HADOOP-8569. CMakeLists.txt: define _GNU_SOURCE and _LARGEFILE_SOURCE.
+    (Colin Patrick McCabe via atm)
+
+    HADOOP-9323. Fix typos in API documentation. (suresh)
+
+    HADOOP-7487. DF should throw a more reasonable exception when mount cannot
+    be determined. (Andrew Wang via atm)
+
+    HADOOP-8917. add LOCALE.US to toLowerCase in SecurityUtil.replacePattern.
+    (Arpit Gupta via suresh)
+
+    HADOOP-9342. Remove jline from distribution. (thw via tucu)
+
+    HADOOP-9230. TestUniformSizeInputFormat fails intermittently. 
+    (kkambatl via tucu)
+
+    HADOOP-9349. Confusing output when running hadoop version from one hadoop 
+    installation when HADOOP_HOME points to another. (sandyr via tucu)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -1351,7 +1378,10 @@ Release 0.23.7 - UNRELEASED
     permissions (Ivan A. Veselovsky via bobby)
 
     HADOOP-9067. provide test for LocalFileSystem.reportChecksumFailure
-    (Ivan A. Veselovsky via bobby) 
+    (Ivan A. Veselovsky via bobby)
+
+    HADOOP-9336. Allow UGI of current connection to be queried. (Daryn Sharp
+    via kihwal)
 
   OPTIMIZATIONS
 
@@ -1363,6 +1393,9 @@ Release 0.23.7 - UNRELEASED
     HADOOP-9303. command manual dfsadmin missing entry for restoreFailedStorage
     option (Andy Isaacson via tgraves)
 
+    HADOOP-9339. IPC.Server incorrectly sets UGI auth type (Daryn Sharp via 
+    kihwal)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 4 - 0
hadoop-common-project/hadoop-common/pom.xml

@@ -219,6 +219,10 @@
       <artifactId>zookeeper</artifactId>
       <version>3.4.2</version>
       <exclusions>
+        <exclusion>
+          <groupId>jline</groupId>
+          <artifactId>jline</artifactId>
+        </exclusion>
         <exclusion>
           <!-- otherwise seems to drag in junit 3.8.1 via jline -->
           <groupId>junit</groupId>

+ 2 - 1
hadoop-common-project/hadoop-common/src/CMakeLists.txt

@@ -90,7 +90,8 @@ find_package(ZLIB REQUIRED)
 SET(CMAKE_FIND_LIBRARY_SUFFIXES STORED_CMAKE_FIND_LIBRARY_SUFFIXES)
 
 set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
 set(D main/native/src/org/apache/hadoop)
 set(T main/native/src/test/org/apache/hadoop)
 

+ 6 - 0
hadoop-common-project/hadoop-common/src/main/bin/hadoop

@@ -50,6 +50,12 @@ fi
 
 COMMAND=$1
 case $COMMAND in
+  # usage flags
+  --help|-help|-h)
+    print_usage
+    exit
+    ;;
+
   #hdfs commands
   namenode|secondarynamenode|datanode|dfs|dfsadmin|fsck|balancer|fetchdt|oiv|dfsgroups)
     echo "DEPRECATED: Use of this script to execute hdfs command is deprecated." 1>&2

+ 1 - 19
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java

@@ -21,8 +21,6 @@ package org.apache.hadoop.fs;
 import java.io.*;
 import java.util.Arrays;
 
-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.conf.Configuration;
@@ -32,7 +30,7 @@ import org.apache.hadoop.util.PureJavaCrc32;
 
 /****************************************************************
  * Abstract Checksumed FileSystem.
- * It provide a basice implementation of a Checksumed FileSystem,
+ * It provide a basic implementation of a Checksumed FileSystem,
  * which creates a checksum file for each raw file.
  * It generates & verifies checksums at the client side.
  *
@@ -118,9 +116,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
    * It verifies that data matches checksums.
    *******************************************************/
   private static class ChecksumFSInputChecker extends FSInputChecker {
-    public static final Log LOG 
-      = LogFactory.getLog(FSInputChecker.class);
-    
     private ChecksumFileSystem fs;
     private FSDataInputStream datas;
     private FSDataInputStream sums;
@@ -374,19 +369,6 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
     private FSDataOutputStream sums;
     private static final float CHKSUM_AS_FRACTION = 0.01f;
     
-    public ChecksumFSOutputSummer(ChecksumFileSystem fs, 
-                          Path file, 
-                          boolean overwrite, 
-                          short replication,
-                          long blockSize,
-                          Configuration conf)
-      throws IOException {
-      this(fs, file, overwrite, 
-           conf.getInt(LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY,
-		       LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT),
-           replication, blockSize, null);
-    }
-    
     public ChecksumFSOutputSummer(ChecksumFileSystem fs, 
                           Path file, 
                           boolean overwrite,

+ 78 - 33
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java

@@ -17,19 +17,22 @@
  */
 package org.apache.hadoop.fs;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.BufferedReader;
-
+import java.util.ArrayList;
 import java.util.EnumSet;
+import java.util.NoSuchElementException;
 import java.util.StringTokenizer;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.util.Shell;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /** Filesystem disk space usage statistics.
  * Uses the unix 'df' program to get mount points, and java.io.File for
  * space utilization. Tested on Linux, FreeBSD, Cygwin. */
@@ -44,6 +47,8 @@ public class DF extends Shell {
   private final File dirFile;
   private String filesystem;
   private String mount;
+  
+  private ArrayList<String> output;
 
   enum OSType {
     OS_TYPE_UNIX("UNIX"),
@@ -84,6 +89,7 @@ public class DF extends Shell {
     super(dfInterval);
     this.dirPath = path.getCanonicalPath();
     this.dirFile = new File(this.dirPath);
+    this.output = new ArrayList<String>();
   }
 
   protected OSType getOSType() {
@@ -127,7 +133,21 @@ public class DF extends Shell {
 
   /** @return the filesystem mount point for the indicated volume */
   public String getMount() throws IOException {
+    // Abort early if specified path does not exist
+    if (!dirFile.exists()) {
+      throw new FileNotFoundException("Specified path " + dirFile.getPath()
+          + "does not exist");
+    }
     run();
+    // Skip parsing if df was not successful
+    if (getExitCode() != 0) {
+      StringBuffer sb = new StringBuffer("df could not be run successfully: ");
+      for (String line: output) {
+        sb.append(line);
+      }
+      throw new IOException(sb.toString());
+    }
+    parseOutput();
     return mount;
   }
   
@@ -164,46 +184,71 @@ public class DF extends Shell {
 
   @Override
   protected void parseExecResult(BufferedReader lines) throws IOException {
-    lines.readLine();                         // skip headings
-  
+    output.clear();
     String line = lines.readLine();
-    if (line == null) {
-      throw new IOException( "Expecting a line not the end of stream" );
+    while (line != null) {
+      output.add(line);
+      line = lines.readLine();
     }
+  }
+  
+  @VisibleForTesting
+  protected void parseOutput() throws IOException {
+    if (output.size() < 2) {
+      StringBuffer sb = new StringBuffer("Fewer lines of output than expected");
+      if (output.size() > 0) {
+        sb.append(": " + output.get(0));
+      }
+      throw new IOException(sb.toString());
+    }
+    
+    String line = output.get(1);
     StringTokenizer tokens =
       new StringTokenizer(line, " \t\n\r\f%");
     
-    this.filesystem = tokens.nextToken();
+    try {
+      this.filesystem = tokens.nextToken();
+    } catch (NoSuchElementException e) {
+      throw new IOException("Unexpected empty line");
+    }
     if (!tokens.hasMoreTokens()) {            // for long filesystem name
-      line = lines.readLine();
-      if (line == null) {
-        throw new IOException( "Expecting a line not the end of stream" );
+      if (output.size() > 2) {
+        line = output.get(2);
+      } else {
+        throw new IOException("Expecting additional output after line: "
+            + line);
       }
       tokens = new StringTokenizer(line, " \t\n\r\f%");
     }
 
-    switch(getOSType()) {
-      case OS_TYPE_AIX:
-        Long.parseLong(tokens.nextToken()); // capacity
-        Long.parseLong(tokens.nextToken()); // available
-        Integer.parseInt(tokens.nextToken()); // pct used
-        tokens.nextToken();
-        tokens.nextToken();
-        this.mount = tokens.nextToken();
-        break;
-
-      case OS_TYPE_WIN:
-      case OS_TYPE_SOLARIS:
-      case OS_TYPE_MAC:
-      case OS_TYPE_UNIX:
-      default:
-        Long.parseLong(tokens.nextToken()); // capacity
-        Long.parseLong(tokens.nextToken()); // used
-        Long.parseLong(tokens.nextToken()); // available
-        Integer.parseInt(tokens.nextToken()); // pct used
-        this.mount = tokens.nextToken();
-        break;
-   }
+    try {
+      switch(getOSType()) {
+        case OS_TYPE_AIX:
+          Long.parseLong(tokens.nextToken()); // capacity
+          Long.parseLong(tokens.nextToken()); // available
+          Integer.parseInt(tokens.nextToken()); // pct used
+          tokens.nextToken();
+          tokens.nextToken();
+          this.mount = tokens.nextToken();
+          break;
+
+        case OS_TYPE_WIN:
+        case OS_TYPE_SOLARIS:
+        case OS_TYPE_MAC:
+        case OS_TYPE_UNIX:
+        default:
+          Long.parseLong(tokens.nextToken()); // capacity
+          Long.parseLong(tokens.nextToken()); // used
+          Long.parseLong(tokens.nextToken()); // available
+          Integer.parseInt(tokens.nextToken()); // pct used
+          this.mount = tokens.nextToken();
+          break;
+     }
+    } catch (NoSuchElementException e) {
+      throw new IOException("Could not parse line: " + line);
+    } catch (NumberFormatException e) {
+      throw new IOException("Could not parse line: " + line);
+    }
   }
 
   public static void main(String[] args) throws Exception {

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

@@ -1326,7 +1326,7 @@ public final class FileContext {
    * 
    * 2. Partially qualified URIs (eg scheme but no host)
    * 
-   * fs:///A/B/file  Resolved according to the target file sytem. Eg resolving
+   * fs:///A/B/file  Resolved according to the target file system. Eg resolving
    *                 a symlink to hdfs:///A results in an exception because
    *                 HDFS URIs must be fully qualified, while a symlink to 
    *                 file:///A will not since Hadoop's local file systems 

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

@@ -1864,7 +1864,7 @@ public abstract class FileSystem extends Configured implements Closeable {
    * 
    * Some file systems like LocalFileSystem have an initial workingDir
    * that we use as the starting workingDir. For other file systems
-   * like HDFS there is no built in notion of an inital workingDir.
+   * like HDFS there is no built in notion of an initial workingDir.
    * 
    * @return if there is built in notion of workingDir then it
    * is returned; else a null is returned.

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

@@ -43,7 +43,7 @@ public interface PositionedReadable {
     throws IOException;
   
   /**
-   * Read number of bytes equalt to the length of the buffer, from a given
+   * Read number of bytes equal to the length of the buffer, from a given
    * position within a file. This does not
    * change the current offset of a file, and is thread-safe.
    */

+ 5 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java

@@ -79,19 +79,17 @@ public abstract class TrashPolicy extends Configured {
 
   /**
    * Get an instance of the configured TrashPolicy based on the value 
-   * of the configuration paramater fs.trash.classname.
+   * of the configuration parameter fs.trash.classname.
    *
    * @param conf the configuration to be used
    * @param fs the file system to be used
    * @param home the home directory
    * @return an instance of TrashPolicy
    */
-  public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home)
-      throws IOException {
-    Class<? extends TrashPolicy> trashClass = conf.getClass("fs.trash.classname",
-                                                      TrashPolicyDefault.class,
-                                                      TrashPolicy.class);
-    TrashPolicy trash = (TrashPolicy) ReflectionUtils.newInstance(trashClass, conf);
+  public static TrashPolicy getInstance(Configuration conf, FileSystem fs, Path home) {
+    Class<? extends TrashPolicy> trashClass = conf.getClass(
+        "fs.trash.classname", TrashPolicyDefault.class, TrashPolicy.class);
+    TrashPolicy trash = ReflectionUtils.newInstance(trashClass, conf);
     trash.initialize(conf, fs, home); // initialize TrashPolicy
     return trash;
   }

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 
 /** 
  * A byte sequence that is usable as a key or value.
- * It is resizable and distinguishes between the size of the seqeunce and
+ * It is resizable and distinguishes between the size of the sequence and
  * the current capacity. The hash function is the front of the md5 of the 
  * buffer. The sort order is the same as memcmp.
  */

+ 2 - 2
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java

@@ -128,7 +128,7 @@ public class Text extends BinaryComparable
   /**
    * Returns the Unicode Scalar Value (32-bit integer value)
    * for the character at <code>position</code>. Note that this
-   * method avoids using the converter or doing String instatiation
+   * method avoids using the converter or doing String instantiation
    * @return the Unicode scalar value at position or -1
    *          if the position is invalid or points to a
    *          trailing byte
@@ -527,7 +527,7 @@ public class Text extends BinaryComparable
     int length = 0;
     int state = LEAD_BYTE;
     while (count < start+len) {
-      int aByte = ((int) utf8[count] & 0xFF);
+      int aByte = utf8[count] & 0xFF;
 
       switch (state) {
       case LEAD_BYTE:

+ 8 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java

@@ -313,6 +313,14 @@ public abstract class Server {
     return (addr == null) ? null : addr.getHostAddress();
   }
 
+  /** Returns the RPC remote user when invoked inside an RPC.  Note this
+   *  may be different than the current user if called within another doAs
+   *  @return connection's UGI or null if not an RPC
+   */
+  public static UserGroupInformation getRemoteUser() {
+    Call call = CurCall.get();
+    return (call != null) ? call.connection.user : null;
+  }
  
   /** Return true if the invocation was through an RPC.
    */
@@ -1553,9 +1561,6 @@ public abstract class Server {
       UserGroupInformation protocolUser = ProtoUtil.getUgi(connectionContext);
       if (saslServer == null) {
         user = protocolUser;
-        if (user != null) {
-          user.setAuthenticationMethod(AuthMethod.SIMPLE);
-        }
       } else {
         // user is authenticated
         user.setAuthenticationMethod(authMethod);

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/record/Buffer.java

@@ -192,7 +192,7 @@ public class Buffer implements Comparable, Cloneable {
     int hash = 1;
     byte[] b = this.get();
     for (int i = 0; i < count; i++)
-      hash = (31 * hash) + (int)b[i];
+      hash = (31 * hash) + b[i];
     return hash;
   }
   

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/record/RecordOutput.java

@@ -26,7 +26,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * Interface that alll the serializers have to implement.
+ * Interface that all the serializers have to implement.
  * 
  * @deprecated Replaced by <a href="http://hadoop.apache.org/avro/">Avro</a>.
  */

+ 2 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java

@@ -30,6 +30,7 @@ import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Locale;
 import java.util.ServiceLoader;
 import java.util.Set;
 
@@ -219,7 +220,7 @@ public class SecurityUtil {
     if (fqdn == null || fqdn.isEmpty() || fqdn.equals("0.0.0.0")) {
       fqdn = getLocalHostName();
     }
-    return components[0] + "/" + fqdn.toLowerCase() + "@" + components[2];
+    return components[0] + "/" + fqdn.toLowerCase(Locale.US) + "@" + components[2];
   }
   
   static String getLocalHostName() throws UnknownHostException {

+ 60 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ClassUtil.java

@@ -0,0 +1,60 @@
+/*
+ * 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.util;
+
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.util.Enumeration;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class ClassUtil {
+  /** 
+   * Find a jar that contains a class of the same name, if any.
+   * It will return a jar file, even if that is not the first thing
+   * on the class path that has a class with the same name.
+   * 
+   * @param clazz the class to find.
+   * @return a jar file that contains the class, or null.
+   * @throws IOException
+   */
+  public static String findContainingJar(Class clazz) {
+    ClassLoader loader = clazz.getClassLoader();
+    String classFile = clazz.getName().replaceAll("\\.", "/") + ".class";
+    try {
+      for (Enumeration itr = loader.getResources(classFile);
+          itr.hasMoreElements();) {
+        URL url = (URL) itr.nextElement();
+        if ("jar".equals(url.getProtocol())) {
+          String toReturn = url.getPath();
+          if (toReturn.startsWith("file:")) {
+            toReturn = toReturn.substring("file:".length());
+          }
+          toReturn = URLDecoder.decode(toReturn, "UTF-8");
+          return toReturn.replaceAll("!.*$", "");
+        }
+      }
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    return null;
+  }
+}

+ 7 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/VersionInfo.java

@@ -18,6 +18,11 @@
 
 package org.apache.hadoop.util;
 
+import java.io.IOException;
+import java.net.URL;
+import java.net.URLDecoder;
+import java.util.Enumeration;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -155,5 +160,7 @@ public class VersionInfo {
     System.out.println("Subversion " + getUrl() + " -r " + getRevision());
     System.out.println("Compiled by " + getUser() + " on " + getDate());
     System.out.println("From source with checksum " + getSrcChecksum());
+    System.out.println("This command was run using " + 
+        ClassUtil.findContainingJar(VersionInfo.class));
   }
 }

+ 79 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestDFVariations.java

@@ -17,15 +17,22 @@
 */
 package org.apache.hadoop.fs;
 
-import junit.framework.TestCase;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
+import java.io.BufferedReader;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.StringReader;
 import java.util.EnumSet;
+import java.util.Random;
 
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.Shell;
+import org.junit.Test;
 
-public class TestDFVariations extends TestCase {
+public class TestDFVariations {
 
   public static class XXDF extends DF {
     private final String osName;
@@ -50,6 +57,7 @@ public class TestDFVariations extends TestCase {
     }
   }
 
+  @Test(timeout=5000)
   public void testOSParsing() throws Exception {
     for (DF.OSType ost : EnumSet.allOf(DF.OSType.class)) {
       XXDF df = new XXDF(ost.getId());
@@ -58,6 +66,74 @@ public class TestDFVariations extends TestCase {
         df.getMount());
     }
   }
-
+  
+  @Test(timeout=5000)
+  public void testDFInvalidPath() throws Exception {
+    // Generate a path that doesn't exist
+    Random random = new Random(0xDEADBEEFl);
+    File file = null;
+    byte[] bytes = new byte[64];
+    while (file == null) {
+      random.nextBytes(bytes);
+      final String invalid = new String("/" + bytes);
+      final File invalidFile = new File(invalid);
+      if (!invalidFile.exists()) {
+        file = invalidFile;
+      }
+    }
+    DF df = new DF(file, 0l);
+    try {
+      df.getMount();
+    } catch (FileNotFoundException e) {
+      // expected, since path does not exist
+      GenericTestUtils.assertExceptionContains(file.getName(), e);
+    }
+  }
+  
+  @Test(timeout=5000)
+  public void testDFMalformedOutput() throws Exception {
+    DF df = new DF(new File("/"), 0l);
+    BufferedReader reader = new BufferedReader(new StringReader(
+        "Filesystem     1K-blocks     Used Available Use% Mounted on\n" +
+        "/dev/sda5       19222656 10597036   7649060  59% /"));
+    df.parseExecResult(reader);
+    df.parseOutput();
+    
+    reader = new BufferedReader(new StringReader(
+        "Filesystem     1K-blocks     Used Available Use% Mounted on"));
+    df.parseExecResult(reader);
+    try {
+      df.parseOutput();
+      fail("Expected exception with missing line!");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains(
+          "Fewer lines of output than expected", e);
+      System.out.println(e.toString());
+    }
+    
+    reader = new BufferedReader(new StringReader(
+        "Filesystem     1K-blocks     Used Available Use% Mounted on\n" +
+        " "));
+    df.parseExecResult(reader);
+    try {
+      df.parseOutput();
+      fail("Expected exception with empty line!");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Unexpected empty line", e);
+      System.out.println(e.toString());
+    }
+    
+    reader = new BufferedReader(new StringReader(
+        "Filesystem     1K-blocks     Used Available Use% Mounted on\n" +
+        "       19222656 10597036   7649060  59% /"));
+    df.parseExecResult(reader);
+    try {
+      df.parseOutput();
+      fail("Expected exception with missing field!");
+    } catch (IOException e) {
+      GenericTestUtils.assertExceptionContains("Could not parse line: ", e);
+      System.out.println(e.toString());
+    }
+  }
 }
 

+ 29 - 30
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestDoAsEffectiveUser.java

@@ -127,6 +127,7 @@ public class TestDoAsEffectiveUser {
     public static final long versionID = 1L;
 
     String aMethod() throws IOException;
+    String getServerRemoteUser() throws IOException;
   }
 
   public class TestImpl implements TestProtocol {
@@ -136,6 +137,11 @@ public class TestDoAsEffectiveUser {
       return UserGroupInformation.getCurrentUser().toString();
     }
 
+    @Override
+    public String getServerRemoteUser() throws IOException {
+      return Server.getRemoteUser().toString();
+    }
+    
     @Override
     public long getProtocolVersion(String protocol, long clientVersion)
         throws IOException {
@@ -149,7 +155,23 @@ public class TestDoAsEffectiveUser {
     }
   }
 
-  @Test
+  private void checkRemoteUgi(final Server server,
+      final UserGroupInformation ugi, final Configuration conf)
+          throws Exception {
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws IOException {
+        proxy = RPC.getProxy(
+            TestProtocol.class, TestProtocol.versionID,
+            NetUtils.getConnectAddress(server), conf);
+        Assert.assertEquals(ugi.toString(), proxy.aMethod());
+        Assert.assertEquals(ugi.toString(), proxy.getServerRemoteUser());
+        return null;
+      }
+    });    
+  }
+  
+  @Test(timeout=4000)
   public void testRealUserSetup() throws IOException {
     final Configuration conf = new Configuration();
     conf.setStrings(ProxyUsers
@@ -163,24 +185,13 @@ public class TestDoAsEffectiveUser {
     try {
       server.start();
 
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
+      checkRemoteUgi(server, realUserUgi, conf);
+      
       UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
           PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      String retVal = proxyUserUgi
-          .doAs(new PrivilegedExceptionAction<String>() {
-            @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
-            }
-          });
-
-      Assert.assertEquals(PROXY_USER_NAME + " (auth:SIMPLE) via " + REAL_USER_NAME + " (auth:SIMPLE)", retVal);
+      checkRemoteUgi(server, proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();
@@ -192,7 +203,7 @@ public class TestDoAsEffectiveUser {
     }
   }
 
-  @Test
+  @Test(timeout=4000)
   public void testRealUserAuthorizationSuccess() throws IOException {
     final Configuration conf = new Configuration();
     configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
@@ -206,25 +217,13 @@ public class TestDoAsEffectiveUser {
     try {
       server.start();
 
-      final InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
       UserGroupInformation realUserUgi = UserGroupInformation
           .createRemoteUser(REAL_USER_NAME);
+      checkRemoteUgi(server, realUserUgi, conf);
 
       UserGroupInformation proxyUserUgi = UserGroupInformation
           .createProxyUserForTesting(PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
-      String retVal = proxyUserUgi
-          .doAs(new PrivilegedExceptionAction<String>() {
-            @Override
-            public String run() throws IOException {
-              proxy = RPC.getProxy(TestProtocol.class,
-                  TestProtocol.versionID, addr, conf);
-              String ret = proxy.aMethod();
-              return ret;
-            }
-          });
-
-      Assert.assertEquals(PROXY_USER_NAME + " (auth:SIMPLE) via " + REAL_USER_NAME + " (auth:SIMPLE)", retVal);
+      checkRemoteUgi(server, proxyUserUgi, conf);
     } catch (Exception e) {
       e.printStackTrace();
       Assert.fail();

+ 40 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestClassUtil.java

@@ -0,0 +1,40 @@
+/*
+ * 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.util;
+
+import java.io.File;
+
+import junit.framework.Assert;
+
+import org.apache.log4j.Logger;
+import org.junit.Test;
+
+public class TestClassUtil {
+  @Test(timeout=1000)
+  public void testFindContainingJar() {
+    String containingJar = ClassUtil.findContainingJar(Logger.class);
+    Assert.assertNotNull("Containing jar not found for Logger", 
+        containingJar);
+    File jarFile = new File(containingJar);
+    Assert.assertTrue("Containing jar does not exist on file system", 
+        jarFile.exists());
+    Assert.assertTrue("Incorrect jar file" + containingJar,  
+        jarFile.getName().matches("log4j.+[.]jar"));
+  }
+}

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

@@ -299,6 +299,9 @@ Trunk (Unreleased)
     HDFS-4340. Update addBlock() to inculde inode id as additional argument.
     (Brandon Li via suresh)
 
+    HDFS-4502. JsonUtil.toFileStatus(..) should check if the fileId property
+    exists.  (Brandon Li via suresh)
+
 Release 2.0.4-beta - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -307,6 +310,15 @@ Release 2.0.4-beta - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-4222. NN is unresponsive and loses heartbeats from DNs when 
+    configured to use LDAP and LDAP has issues. (Xiaobo Peng, suresh)
+
+    HDFS-4304. Make FSEditLogOp.MAX_OP_SIZE configurable. (Colin Patrick
+    McCabe via atm)
+
+    HDFS-4518. Finer grained metrics for HDFS capacity.
+    (Arpit Agarwal via suresh)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -321,6 +333,12 @@ Release 2.0.4-beta - UNRELEASED
     but not in dfs.namenode.edits.dir are silently ignored.  (Arpit Agarwal
     via szetszwo)
 
+    HDFS-4482. ReplicationMonitor thread can exit with NPE due to the race 
+    between delete and replication of same file. (umamahesh)
+
+    HDFS-4235. When outputting XML, OfflineEditsViewer can't handle some edits
+    containing non-ASCII strings. (Colin Patrick McCabe via atm)
+
 Release 2.0.3-alpha - 2013-02-06
 
   INCOMPATIBLE CHANGES

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/CMakeLists.txt

@@ -68,8 +68,9 @@ if (NOT GENERATED_JAVAH)
     MESSAGE(FATAL_ERROR "You must set the CMake variable GENERATED_JAVAH")
 endif (NOT GENERATED_JAVAH)
 
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2 -D_GNU_SOURCE")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_GNU_SOURCE")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
 
 include_directories(
     ${GENERATED_JAVAH}

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperEditLogInputStream.java

@@ -163,6 +163,11 @@ class BookKeeperEditLogInputStream extends EditLogInputStream {
     return ("BookKeeperEditLogInputStream {" + this.getName() + "}");
   }
 
+  @Override
+  public void setMaxOpSize(int maxOpSize) {
+    reader.setMaxOpSize(maxOpSize);
+  }
+
   /**
    * Input stream implementation which can be used by 
    * FSEditLogOp.Reader

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -56,6 +56,14 @@ fi
 COMMAND=$1
 shift
 
+case $COMMAND in
+  # usage flags
+  --help|-help|-h)
+    print_usage
+    exit
+    ;;
+esac
+
 # Determine if we're starting a secure datanode, and if so, redefine appropriate variables
 if [ "$COMMAND" == "datanode" ] && [ "$EUID" -eq 0 ] && [ -n "$HADOOP_SECURE_DN_USER" ]; then
   if [ -n "$JSVC_HOME" ]; then

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

@@ -392,6 +392,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_CHECKED_VOLUMES_MINIMUM_DEFAULT = 1;
   public static final String  DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY = "dfs.web.authentication.kerberos.principal";
   public static final String  DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY = "dfs.web.authentication.kerberos.keytab";
+  public static final String  DFS_NAMENODE_MAX_OP_SIZE_KEY = "dfs.namenode.max.op.size";
+  public static final int     DFS_NAMENODE_MAX_OP_SIZE_DEFAULT = 50 * 1024 * 1024;
   
   public static final String DFS_BLOCK_LOCAL_PATH_ACCESS_USER_KEY = "dfs.block.local-path-access.user";
 

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogBackupInputStream.java

@@ -142,4 +142,9 @@ class EditLogBackupInputStream extends EditLogInputStream {
   public boolean isInProgress() {
     return true;
   }
+
+  @Override
+  public void setMaxOpSize(int maxOpSize) {
+    reader.setMaxOpSize(maxOpSize);
+  }
 }

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -32,6 +32,7 @@ import java.security.PrivilegedExceptionAction;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
@@ -53,6 +54,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
   private final long firstTxId;
   private final long lastTxId;
   private final boolean isInProgress;
+  private int maxOpSize;
   static private enum State {
     UNINIT,
     OPEN,
@@ -118,6 +120,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
     this.firstTxId = firstTxId;
     this.lastTxId = lastTxId;
     this.isInProgress = isInProgress;
+    this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
   }
 
   private void init() throws LogHeaderCorruptException, IOException {
@@ -134,6 +137,7 @@ public class EditLogFileInputStream extends EditLogInputStream {
         throw new LogHeaderCorruptException("No header found in log");
       }
       reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
+      reader.setMaxOpSize(maxOpSize);
       state = State.OPEN;
     } finally {
       if (reader == null) {
@@ -412,5 +416,12 @@ public class EditLogFileInputStream extends EditLogInputStream {
       return url.toString();
     }
   }
-  
+
+  @Override
+  public void setMaxOpSize(int maxOpSize) {
+    this.maxOpSize = maxOpSize;
+    if (reader != null) {
+      reader.setMaxOpSize(maxOpSize);
+    }
+  }
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogInputStream.java

@@ -165,4 +165,9 @@ public abstract class EditLogInputStream implements Closeable {
    * Return true if this stream is in progress, false if it is finalized.
    */
   public abstract boolean isInProgress();
+  
+  /**
+   * Set the maximum opcode size in bytes.
+   */
+  public abstract void setMaxOpSize(int maxOpSize);
 }

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -1343,6 +1343,11 @@ public class FSDirectory implements Closeable {
 
     // fill up the inodes in the path from this inode to root
     for (int i = 0; i < depth; i++) {
+      if (inode == null) {
+        NameNode.stateChangeLog.warn("Could not get full path."
+            + " Corresponding file might have deleted already.");
+        return null;
+      }
       inodes[depth-i-1] = inode;
       inode = inode.parent;
     }

+ 11 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -51,6 +51,7 @@ import org.apache.hadoop.io.WritableFactory;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.util.XMLUtils.Stanza;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DeprecatedUTF8;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
@@ -75,11 +76,6 @@ import java.io.EOFException;
 public abstract class FSEditLogOp {
   public final FSEditLogOpCodes opCode;
   long txid;
-  /**
-   * Opcode size is limited to 1.5 megabytes
-   */
-  public static final int MAX_OP_SIZE = (3 * 1024 * 1024) / 2;
-
 
   @SuppressWarnings("deprecation")
   final public static class OpInstanceCache {
@@ -2246,6 +2242,7 @@ public abstract class FSEditLogOp {
     private final int logVersion;
     private final Checksum checksum;
     private final OpInstanceCache cache;
+    private int maxOpSize;
 
     /**
      * Construct the reader
@@ -2253,7 +2250,8 @@ public abstract class FSEditLogOp {
      * @param logVersion The version of the data coming from the stream.
      */
     @SuppressWarnings("deprecation")
-    public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
+    public Reader(DataInputStream in, StreamLimiter limiter,
+        int logVersion) {
       this.logVersion = logVersion;
       if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
         this.checksum = new PureJavaCrc32();
@@ -2269,6 +2267,11 @@ public abstract class FSEditLogOp {
       }
       this.limiter = limiter;
       this.cache = new OpInstanceCache();
+      this.maxOpSize = DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
+    }
+
+    public void setMaxOpSize(int maxOpSize) {
+      this.maxOpSize = maxOpSize;
     }
 
     /**
@@ -2363,8 +2366,8 @@ public abstract class FSEditLogOp {
      * problematic byte.  This usually means the beginning of the opcode.
      */
     private FSEditLogOp decodeOp() throws IOException {
-      limiter.setLimit(MAX_OP_SIZE);
-      in.mark(MAX_OP_SIZE);
+      limiter.setLimit(maxOpSize);
+      in.mark(maxOpSize);
 
       if (checksum != null) {
         checksum.reset();

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -607,6 +607,12 @@ public class FSImage implements Closeable {
       editStreams = FSImagePreTransactionalStorageInspector
         .getEditLogStreams(storage);
     }
+    int maxOpSize = conf.getInt(DFSConfigKeys.
+          DFS_NAMENODE_MAX_OP_SIZE_KEY,
+        DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT);
+    for (EditLogInputStream elis : editStreams) {
+      elis.setMaxOpSize(maxOpSize);
+    }
  
     LOG.debug("Planning to load image :\n" + imageFile);
     for (EditLogInputStream l : editStreams) {

+ 129 - 107
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -300,6 +300,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private final boolean isPermissionEnabled;
   private final boolean persistBlocks;
   private final UserGroupInformation fsOwner;
+  private final String fsOwnerShortUserName;
   private final String supergroup;
   private final boolean standbyShouldCheckpoint;
   
@@ -529,6 +530,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       this.datanodeStatistics = blockManager.getDatanodeManager().getDatanodeStatistics();
 
       this.fsOwner = UserGroupInformation.getCurrentUser();
+      this.fsOwnerShortUserName = fsOwner.getShortUserName();
       this.supergroup = conf.get(DFS_PERMISSIONS_SUPERUSERGROUP_KEY, 
                                  DFS_PERMISSIONS_SUPERUSERGROUP_DEFAULT);
       this.isPermissionEnabled = conf.getBoolean(DFS_PERMISSIONS_ENABLED_KEY,
@@ -1111,9 +1113,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * Dump all metadata into specified file
    */
   void metaSave(String filename) throws IOException {
+    checkSuperuserPrivilege();
     writeLock();
     try {
-      checkSuperuserPrivilege();
       File file = new File(System.getProperty("hadoop.log.dir"), filename);
       PrintWriter out = new PrintWriter(new BufferedWriter(
           new OutputStreamWriter(new FileOutputStream(file, true), Charsets.UTF_8)));
@@ -1190,6 +1192,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, FileNotFoundException, SafeModeException,
       UnresolvedLinkException, IOException {
     HdfsFileStatus resultingStat = null;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -1197,7 +1200,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set permission for " + src, safeMode);
       }
-      checkOwner(src);
+      checkOwner(pc, src);
       dir.setPermission(src, permission);
       if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(src, false);
@@ -1236,6 +1239,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, FileNotFoundException, SafeModeException,
       UnresolvedLinkException, IOException {
     HdfsFileStatus resultingStat = null;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -1243,14 +1247,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set owner for " + src, safeMode);
       }
-      FSPermissionChecker pc = checkOwner(src);
-      if (!pc.isSuper) {
-        if (username != null && !pc.user.equals(username)) {
-          throw new AccessControlException("Non-super user cannot change owner.");
+      checkOwner(pc, src);
+      if (!pc.isSuperUser()) {
+        if (username != null && !pc.getUser().equals(username)) {
+          throw new AccessControlException("Non-super user cannot change owner");
         }
         if (group != null && !pc.containsGroup(group)) {
-          throw new AccessControlException("User does not belong to " + group
-            + " .");
+          throw new AccessControlException("User does not belong to " + group);
         }
       }
       dir.setOwner(src, username, group);
@@ -1300,8 +1303,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   LocatedBlocks getBlockLocations(String src, long offset, long length,
       boolean doAccessTime, boolean needBlockToken, boolean checkSafeMode)
       throws FileNotFoundException, UnresolvedLinkException, IOException {
+    FSPermissionChecker pc = getPermissionChecker();
     try {
-      return getBlockLocationsInt(src, offset, length, doAccessTime,
+      return getBlockLocationsInt(pc, src, offset, length, doAccessTime,
                                   needBlockToken, checkSafeMode);
     } catch (AccessControlException e) {
       if (isAuditEnabled() && isExternalInvocation()) {
@@ -1313,11 +1317,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  private LocatedBlocks getBlockLocationsInt(String src, long offset, long length,
-      boolean doAccessTime, boolean needBlockToken, boolean checkSafeMode)
+  private LocatedBlocks getBlockLocationsInt(FSPermissionChecker pc,
+      String src, long offset, long length, boolean doAccessTime,
+      boolean needBlockToken, boolean checkSafeMode)
       throws FileNotFoundException, UnresolvedLinkException, IOException {
     if (isPermissionEnabled) {
-      checkPathAccess(src, FsAction.READ);
+      checkPathAccess(pc, src, FsAction.READ);
     }
 
     if (offset < 0) {
@@ -1447,13 +1452,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
 
     HdfsFileStatus resultingStat = null;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot concat " + target, safeMode);
       }
-      concatInternal(target, srcs);
+      concatInternal(pc, target, srcs);
       if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(target, false);
       }
@@ -1469,18 +1475,18 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   /** See {@link #concat(String, String[])} */
-  private void concatInternal(String target, String [] srcs) 
+  private void concatInternal(FSPermissionChecker pc, String target, String [] srcs) 
       throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
 
     // write permission for the target
     if (isPermissionEnabled) {
-      checkPathAccess(target, FsAction.WRITE);
+      checkPathAccess(pc, target, FsAction.WRITE);
 
       // and srcs
       for(String aSrc: srcs) {
-        checkPathAccess(aSrc, FsAction.READ); // read the file
-        checkParentAccess(aSrc, FsAction.WRITE); // for delete 
+        checkPathAccess(pc, aSrc, FsAction.READ); // read the file
+        checkParentAccess(pc, aSrc, FsAction.WRITE); // for delete 
       }
     }
 
@@ -1597,13 +1603,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new IOException("Access time for hdfs is not configured. " +
                             " Please set " + DFS_NAMENODE_ACCESSTIME_PRECISION_KEY + " configuration parameter.");
     }
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
 
       // Write access is required to set access and modification times
       if (isPermissionEnabled) {
-        checkPathAccess(src, FsAction.WRITE);
+        checkPathAccess(pc, src, FsAction.WRITE);
       }
       INode inode = dir.getINode(src);
       if (inode != null) {
@@ -1644,6 +1651,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
     HdfsFileStatus resultingStat = null;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -1651,7 +1659,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       if (!createParent) {
         verifyParentDir(link);
       }
-      createSymlinkInternal(target, link, dirPerms, createParent);
+      createSymlinkInternal(pc, target, link, dirPerms, createParent);
       if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(link, false);
       }
@@ -1669,8 +1677,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Create a symbolic link.
    */
-  private void createSymlinkInternal(String target, String link,
-      PermissionStatus dirPerms, boolean createParent)
+  private void createSymlinkInternal(FSPermissionChecker pc, String target,
+      String link, PermissionStatus dirPerms, boolean createParent)
       throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
@@ -1688,7 +1696,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           +" either because the filename is invalid or the file exists");
     }
     if (isPermissionEnabled) {
-      checkAncestorAccess(link, FsAction.WRITE);
+      checkAncestorAccess(pc, link, FsAction.WRITE);
     }
     // validate that we have enough inodes.
     checkFsObjectLimit();
@@ -1727,17 +1735,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   private boolean setReplicationInt(final String src, final short replication)
       throws IOException {
     blockManager.verifyReplication(src, replication, null);
-
     final boolean isFile;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set replication for " + src, safeMode);
       }
       if (isPermissionEnabled) {
-        checkPathAccess(src, FsAction.WRITE);
+        checkPathAccess(pc, src, FsAction.WRITE);
       }
 
       final short[] oldReplication = new short[1];
@@ -1761,11 +1768,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   long getPreferredBlockSize(String filename) 
       throws IOException, UnresolvedLinkException {
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
       if (isPermissionEnabled) {
-        checkTraverse(filename);
+        checkTraverse(pc, filename);
       }
       return dir.getPreferredBlockSize(filename);
     } finally {
@@ -1826,11 +1834,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       FileNotFoundException, ParentNotDirectoryException, IOException {
     boolean skipSync = false;
     final HdfsFileStatus stat;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
-      startFileInternal(src, permissions, holder, clientMachine, flag,
+      startFileInternal(pc, src, permissions, holder, clientMachine, flag,
           createParent, replication, blockSize);
       stat = dir.getFileInfo(src, false);
     } catch (StandbyException se) {
@@ -1869,7 +1877,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * 
    * @return the last block locations if the block is partial or null otherwise
    */
-  private LocatedBlock startFileInternal(String src,
+  private LocatedBlock startFileInternal(FSPermissionChecker pc, String src,
       PermissionStatus permissions, String holder, String clientMachine,
       EnumSet<CreateFlag> flag, boolean createParent, short replication,
       long blockSize) throws SafeModeException, FileAlreadyExistsException,
@@ -1902,9 +1910,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     boolean append = flag.contains(CreateFlag.APPEND);
     if (isPermissionEnabled) {
       if (append || (overwrite && pathExists)) {
-        checkPathAccess(src, FsAction.WRITE);
+        checkPathAccess(pc, src, FsAction.WRITE);
       } else {
-        checkAncestorAccess(src, FsAction.WRITE);
+        checkAncestorAccess(pc, src, FsAction.WRITE);
       }
     }
 
@@ -2027,6 +2035,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   boolean recoverLease(String src, String holder, String clientMachine)
       throws IOException {
     boolean skipSync = false;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2044,7 +2053,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         return true;
       }
       if (isPermissionEnabled) {
-        checkPathAccess(src, FsAction.WRITE);
+        checkPathAccess(pc, src, FsAction.WRITE);
       }
   
       recoverLeaseInternal(inode, src, holder, clientMachine, true);
@@ -2167,11 +2176,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
           DFS_SUPPORT_APPEND_KEY + " configuration option to enable it.");
     }
     LocatedBlock lb = null;
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
 
-      lb = startFileInternal(src, null, holder, clientMachine, 
+      lb = startFileInternal(pc, src, null, holder, clientMachine, 
                         EnumSet.of(CreateFlag.APPEND), 
                         false, blockManager.maxReplication, 0);
     } catch (StandbyException se) {
@@ -2708,11 +2718,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src +
           " to " + dst);
     }
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
 
-      status = renameToInternal(src, dst);
+      status = renameToInternal(pc, src, dst);
       if (status && isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false);
       }
@@ -2730,7 +2741,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   /** @deprecated See {@link #renameTo(String, String)} */
   @Deprecated
-  private boolean renameToInternal(String src, String dst)
+  private boolean renameToInternal(FSPermissionChecker pc, String src, String dst)
     throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
     if (isInSafeMode()) {
@@ -2746,8 +2757,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       //      of rewriting the dst
       String actualdst = dir.isDir(dst)?
           dst + Path.SEPARATOR + new Path(src).getName(): dst;
-      checkParentAccess(src, FsAction.WRITE);
-      checkAncestorAccess(actualdst, FsAction.WRITE);
+      checkParentAccess(pc, src, FsAction.WRITE);
+      checkAncestorAccess(pc, actualdst, FsAction.WRITE);
     }
 
     if (dir.renameTo(src, dst)) {
@@ -2765,11 +2776,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: with options - "
           + src + " to " + dst);
     }
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
-      renameToInternal(src, dst, options);
+      renameToInternal(pc, src, dst, options);
       if (isAuditEnabled() && isExternalInvocation()) {
         resultingStat = dir.getFileInfo(dst, false); 
       }
@@ -2787,7 +2798,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     }
   }
 
-  private void renameToInternal(String src, String dst,
+  private void renameToInternal(FSPermissionChecker pc, String src, String dst,
       Options.Rename... options) throws IOException {
     assert hasWriteLock();
     if (isInSafeMode()) {
@@ -2797,8 +2808,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new InvalidPathException("Invalid name: " + dst);
     }
     if (isPermissionEnabled) {
-      checkParentAccess(src, FsAction.WRITE);
-      checkAncestorAccess(dst, FsAction.WRITE);
+      checkParentAccess(pc, src, FsAction.WRITE);
+      checkAncestorAccess(pc, dst, FsAction.WRITE);
     }
 
     dir.renameTo(src, dst, options);
@@ -2840,6 +2851,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return status;
   }
     
+  private FSPermissionChecker getPermissionChecker()
+      throws AccessControlException {
+    return new FSPermissionChecker(fsOwnerShortUserName, supergroup);
+  }
   /**
    * Remove a file/directory from the namespace.
    * <p>
@@ -2856,7 +2871,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throws AccessControlException, SafeModeException, UnresolvedLinkException,
              IOException {
     BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
-
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
@@ -2867,7 +2882,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new IOException(src + " is non empty");
       }
       if (enforcePermission && isPermissionEnabled) {
-        checkPermission(src, false, null, FsAction.WRITE, null, FsAction.ALL);
+        checkPermission(pc, src, false, null, FsAction.WRITE, null, FsAction.ALL);
       }
       // Unlink the target directory from directory tree
       if (!dir.delete(src, collectedBlocks)) {
@@ -2984,9 +2999,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     throws AccessControlException, UnresolvedLinkException,
            StandbyException, IOException {
     HdfsFileStatus stat = null;
-
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
-
     try {
       checkOperation(OperationCategory.READ);
 
@@ -2994,7 +3008,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throw new InvalidPathException("Invalid file name: " + src);
       }
       if (isPermissionEnabled) {
-        checkTraverse(src);
+        checkTraverse(pc, src);
       }
       stat = dir.getFileInfo(src, resolveLink);
     } catch (AccessControlException e) {
@@ -3038,11 +3052,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     }
+    FSPermissionChecker pc = getPermissionChecker();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-
-      status = mkdirsInternal(src, permissions, createParent);
+      status = mkdirsInternal(pc, src, permissions, createParent);
     } finally {
       writeUnlock();
     }
@@ -3059,7 +3073,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Create all the necessary directories
    */
-  private boolean mkdirsInternal(String src,
+  private boolean mkdirsInternal(FSPermissionChecker pc, String src,
       PermissionStatus permissions, boolean createParent) 
       throws IOException, UnresolvedLinkException {
     assert hasWriteLock();
@@ -3067,7 +3081,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new SafeModeException("Cannot create directory " + src, safeMode);
     }
     if (isPermissionEnabled) {
-      checkTraverse(src);
+      checkTraverse(pc, src);
     }
     if (dir.isDir(src)) {
       // all the users of mkdirs() are used to expect 'true' even if
@@ -3078,7 +3092,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       throw new InvalidPathException(src);
     }
     if (isPermissionEnabled) {
-      checkAncestorAccess(src, FsAction.WRITE);
+      checkAncestorAccess(pc, src, FsAction.WRITE);
     }
     if (!createParent) {
       verifyParentDir(src);
@@ -3097,12 +3111,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
 
   ContentSummary getContentSummary(String src) throws AccessControlException,
       FileNotFoundException, UnresolvedLinkException, StandbyException {
+    FSPermissionChecker pc = new FSPermissionChecker(fsOwnerShortUserName,
+        supergroup);
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-
       if (isPermissionEnabled) {
-        checkPermission(src, false, null, null, null, FsAction.READ_EXECUTE);
+        checkPermission(pc, src, false, null, null, null, FsAction.READ_EXECUTE);
       }
       return dir.getContentSummary(src);
     } finally {
@@ -3117,15 +3132,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   void setQuota(String path, long nsQuota, long dsQuota) 
       throws IOException, UnresolvedLinkException {
+    checkSuperuserPrivilege();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       if (isInSafeMode()) {
         throw new SafeModeException("Cannot set quota on " + path, safeMode);
       }
-      if (isPermissionEnabled) {
-        checkSuperuserPrivilege();
-      }
       dir.setQuota(path, nsQuota, dsQuota);
     } finally {
       writeUnlock();
@@ -3494,15 +3507,16 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       boolean needLocation) 
     throws AccessControlException, UnresolvedLinkException, IOException {
     DirectoryListing dl;
+    FSPermissionChecker pc = getPermissionChecker();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
 
       if (isPermissionEnabled) {
         if (dir.isDir(src)) {
-          checkPathAccess(src, FsAction.READ_EXECUTE);
+          checkPathAccess(pc, src, FsAction.READ_EXECUTE);
         } else {
-          checkTraverse(src);
+          checkTraverse(pc, src);
         }
       }
       if (isAuditEnabled() && isExternalInvocation()) {
@@ -3721,42 +3735,49 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return stats;
   }
 
-  /**
-   * Total raw bytes including non-dfs used space.
-   */
   @Override // FSNamesystemMBean
+  @Metric({"CapacityTotal",
+      "Total raw capacity of data nodes in bytes"})
   public long getCapacityTotal() {
     return datanodeStatistics.getCapacityTotal();
   }
 
-  @Metric
+  @Metric({"CapacityTotalGB",
+      "Total raw capacity of data nodes in GB"})
   public float getCapacityTotalGB() {
     return DFSUtil.roundBytesToGB(getCapacityTotal());
   }
 
-  /**
-   * Total used space by data nodes
-   */
   @Override // FSNamesystemMBean
+  @Metric({"CapacityUsed",
+      "Total used capacity across all data nodes in bytes"})
   public long getCapacityUsed() {
     return datanodeStatistics.getCapacityUsed();
   }
 
-  @Metric
+  @Metric({"CapacityUsedGB",
+      "Total used capacity across all data nodes in GB"})
   public float getCapacityUsedGB() {
     return DFSUtil.roundBytesToGB(getCapacityUsed());
   }
 
-  @Override
+  @Override // FSNamesystemMBean
+  @Metric({"CapacityRemaining", "Remaining capacity in bytes"})
   public long getCapacityRemaining() {
     return datanodeStatistics.getCapacityRemaining();
   }
 
-  @Metric
+  @Metric({"CapacityRemainingGB", "Remaining capacity in GB"})
   public float getCapacityRemainingGB() {
     return DFSUtil.roundBytesToGB(getCapacityRemaining());
   }
 
+  @Metric({"CapacityUsedNonDFS",
+      "Total space used by data nodes for non DFS purposes in bytes"})
+  public long getCapacityUsedNonDFS() {
+    return datanodeStatistics.getCapacityUsedNonDFS();
+  }
+
   /**
    * Total number of connections.
    */
@@ -3803,9 +3824,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws IOException if 
    */
   void saveNamespace() throws AccessControlException, IOException {
+    checkSuperuserPrivilege();
     readLock();
     try {
-      checkSuperuserPrivilege();
       if (!isInSafeMode()) {
         throw new IOException("Safe mode should be turned ON " +
                               "in order to create namespace image.");
@@ -3824,9 +3845,9 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    * @throws AccessControlException if superuser privilege is violated.
    */
   boolean restoreFailedStorage(String arg) throws AccessControlException {
+    checkSuperuserPrivilege();
     writeLock();
     try {
-      checkSuperuserPrivilege();
       
       // if it is disabled - enable it and vice versa.
       if(arg.equals("check"))
@@ -3846,10 +3867,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
     
   void finalizeUpgrade() throws IOException {
+    checkSuperuserPrivilege();
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
-      checkSuperuserPrivilege();
       getFSImage().finalizeUpgrade();
     } finally {
       writeUnlock();
@@ -4585,10 +4606,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
 
   CheckpointSignature rollEditLog() throws IOException {
+    checkSuperuserPrivilege();
     writeLock();
     try {
       checkOperation(OperationCategory.JOURNAL);
-      checkSuperuserPrivilege();
       if (isInSafeMode()) {
         throw new SafeModeException("Log not rolled", safeMode);
       }
@@ -4639,61 +4660,64 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  private FSPermissionChecker checkOwner(String path
-      ) throws AccessControlException, UnresolvedLinkException {
-    return checkPermission(path, true, null, null, null, null);
+  private void checkOwner(FSPermissionChecker pc, String path)
+      throws AccessControlException, UnresolvedLinkException {
+    checkPermission(pc, path, true, null, null, null, null);
   }
 
-  private FSPermissionChecker checkPathAccess(String path, FsAction access
-      ) throws AccessControlException, UnresolvedLinkException {
-    return checkPermission(path, false, null, null, access, null);
+  private void checkPathAccess(FSPermissionChecker pc,
+      String path, FsAction access) throws AccessControlException,
+      UnresolvedLinkException {
+    checkPermission(pc, path, false, null, null, access, null);
   }
 
-  private FSPermissionChecker checkParentAccess(String path, FsAction access
-      ) throws AccessControlException, UnresolvedLinkException {
-    return checkPermission(path, false, null, access, null, null);
+  private void checkParentAccess(FSPermissionChecker pc,
+      String path, FsAction access) throws AccessControlException,
+      UnresolvedLinkException {
+    checkPermission(pc, path, false, null, access, null, null);
   }
 
-  private FSPermissionChecker checkAncestorAccess(String path, FsAction access
-      ) throws AccessControlException, UnresolvedLinkException {
-    return checkPermission(path, false, access, null, null, null);
+  private void checkAncestorAccess(FSPermissionChecker pc,
+      String path, FsAction access) throws AccessControlException,
+      UnresolvedLinkException {
+    checkPermission(pc, path, false, access, null, null, null);
   }
 
-  private FSPermissionChecker checkTraverse(String path
-      ) throws AccessControlException, UnresolvedLinkException {
-    return checkPermission(path, false, null, null, null, null);
+  private void checkTraverse(FSPermissionChecker pc, String path)
+      throws AccessControlException, UnresolvedLinkException {
+    checkPermission(pc, path, false, null, null, null, null);
   }
 
   @Override
-  public void checkSuperuserPrivilege() throws AccessControlException {
+  public void checkSuperuserPrivilege()
+      throws AccessControlException {
     if (isPermissionEnabled) {
-      FSPermissionChecker.checkSuperuserPrivilege(fsOwner, supergroup);
+      FSPermissionChecker pc = getPermissionChecker();
+      pc.checkSuperuserPrivilege();
     }
   }
 
   /**
-   * Check whether current user have permissions to access the path.
-   * For more details of the parameters, see
-   * {@link FSPermissionChecker#checkPermission(String, INodeDirectory, boolean, FsAction, FsAction, FsAction, FsAction)}.
+   * Check whether current user have permissions to access the path. For more
+   * details of the parameters, see
+   * {@link FSPermissionChecker#checkPermission()}.
    */
-  private FSPermissionChecker checkPermission(String path, boolean doCheckOwner,
-      FsAction ancestorAccess, FsAction parentAccess, FsAction access,
-      FsAction subAccess) throws AccessControlException, UnresolvedLinkException {
-    FSPermissionChecker pc = new FSPermissionChecker(
-        fsOwner.getShortUserName(), supergroup);
-    if (!pc.isSuper) {
+  private void checkPermission(FSPermissionChecker pc,
+      String path, boolean doCheckOwner, FsAction ancestorAccess,
+      FsAction parentAccess, FsAction access, FsAction subAccess)
+      throws AccessControlException, UnresolvedLinkException {
+    if (!pc.isSuperUser()) {
       dir.waitForReady();
       readLock();
       try {
-        pc.checkPermission(path, dir.rootDir, doCheckOwner,
-            ancestorAccess, parentAccess, access, subAccess);
+        pc.checkPermission(path, dir.rootDir, doCheckOwner, ancestorAccess,
+            parentAccess, access, subAccess);
       } finally {
         readUnlock();
-      } 
+      }
     }
-    return pc;
   }
-
+  
   /**
    * Check to see if we have exceeded the limit on the number
    * of inodes.
@@ -5137,16 +5161,14 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
    */
   Collection<CorruptFileBlockInfo> listCorruptFileBlocks(String path,
 	String[] cookieTab) throws IOException {
-
+    checkSuperuserPrivilege();
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-
       if (!isPopulatingReplQueues()) {
         throw new IOException("Cannot run listCorruptFileBlocks because " +
                               "replication queues have not been initialized.");
       }
-      checkSuperuserPrivilege();
       // print a limited # of corrupt files per call
       int count = 0;
       ArrayList<CorruptFileBlockInfo> corruptFiles = new ArrayList<CorruptFileBlockInfo>();

+ 37 - 19
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.Stack;
@@ -31,14 +32,20 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 
-/** Perform permission checking in {@link FSNamesystem}. */
+/** 
+ * Class that helps in checking file system permission.
+ * The state of this class need not be synchronized as it has data structures that
+ * are read-only.
+ * 
+ * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
+ */
 class FSPermissionChecker {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
-
   private final UserGroupInformation ugi;
-  public final String user;
-  private final Set<String> groups = new HashSet<String>();
-  public final boolean isSuper;
+  private final String user;  
+  /** A set with group namess. Not synchronized since it is unmodifiable */
+  private final Set<String> groups;
+  private final boolean isSuper;
   
   FSPermissionChecker(String fsOwner, String supergroup
       ) throws AccessControlException{
@@ -47,10 +54,9 @@ class FSPermissionChecker {
     } catch (IOException e) {
       throw new AccessControlException(e); 
     } 
-    
-    groups.addAll(Arrays.asList(ugi.getGroupNames()));
+    HashSet<String> s = new HashSet<String>(Arrays.asList(ugi.getGroupNames()));
+    groups = Collections.unmodifiableSet(s);
     user = ugi.getShortUserName();
-    
     isSuper = user.equals(fsOwner) || groups.contains(supergroup);
   }
 
@@ -60,20 +66,23 @@ class FSPermissionChecker {
    */
   public boolean containsGroup(String group) {return groups.contains(group);}
 
+  public String getUser() {
+    return user;
+  }
+  
+  public boolean isSuperUser() {
+    return isSuper;
+  }
+  
   /**
    * Verify if the caller has the required permission. This will result into 
    * an exception if the caller is not allowed to access the resource.
-   * @param owner owner of the system
-   * @param supergroup supergroup of the system
    */
-  public static void checkSuperuserPrivilege(UserGroupInformation owner, 
-                                             String supergroup) 
-                     throws AccessControlException {
-    FSPermissionChecker checker = 
-      new FSPermissionChecker(owner.getShortUserName(), supergroup);
-    if (!checker.isSuper) {
+  public void checkSuperuserPrivilege()
+      throws AccessControlException {
+    if (!isSuper) {
       throw new AccessControlException("Access denied for user " 
-          + checker.user + ". Superuser privilege is required");
+          + user + ". Superuser privilege is required");
     }
   }
   
@@ -103,9 +112,11 @@ class FSPermissionChecker {
    * @param subAccess If path is a directory,
    * it is the access required of the path and all the sub-directories.
    * If path is not a directory, there is no effect.
-   * @return a PermissionChecker object which caches data for later use.
    * @throws AccessControlException
    * @throws UnresolvedLinkException
+   * 
+   * Guarded by {@link FSNamesystem#readLock()}
+   * Caller of this method must hold that lock.
    */
   void checkPermission(String path, INodeDirectory root, boolean doCheckOwner,
       FsAction ancestorAccess, FsAction parentAccess, FsAction access,
@@ -148,6 +159,7 @@ class FSPermissionChecker {
       }
   }
 
+  /** Guarded by {@link FSNamesystem#readLock()} */
   private void checkOwner(INode inode) throws AccessControlException {
     if (inode != null && user.equals(inode.getUserName())) {
       return;
@@ -155,6 +167,7 @@ class FSPermissionChecker {
     throw new AccessControlException("Permission denied");
   }
 
+  /** Guarded by {@link FSNamesystem#readLock()} */
   private void checkTraverse(INode[] inodes, int last
       ) throws AccessControlException {
     for(int j = 0; j <= last; j++) {
@@ -162,6 +175,7 @@ class FSPermissionChecker {
     }
   }
 
+  /** Guarded by {@link FSNamesystem#readLock()} */
   private void checkSubAccess(INode inode, FsAction access
       ) throws AccessControlException {
     if (inode == null || !inode.isDirectory()) {
@@ -181,11 +195,13 @@ class FSPermissionChecker {
     }
   }
 
+  /** Guarded by {@link FSNamesystem#readLock()} */
   private void check(INode[] inodes, int i, FsAction access
       ) throws AccessControlException {
     check(i >= 0? inodes[i]: null, access);
   }
 
+  /** Guarded by {@link FSNamesystem#readLock()} */
   private void check(INode inode, FsAction access
       ) throws AccessControlException {
     if (inode == null) {
@@ -206,7 +222,9 @@ class FSPermissionChecker {
         + ", access=" + access + ", inode=" + inode);
   }
 
-  private void checkStickyBit(INode parent, INode inode) throws AccessControlException {
+  /** Guarded by {@link FSNamesystem#readLock()} */
+  private void checkStickyBit(INode parent, INode inode)
+      throws AccessControlException {
     if(!parent.getFsPermission().getStickyBit()) {
       return;
     }

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java

@@ -282,7 +282,11 @@ abstract class INode implements Comparable<byte[]> {
 
   String getLocalParentDir() {
     INode inode = isRoot() ? this : getParent();
-    return (inode != null) ? inode.getFullPathName() : "";
+    String parentDir = "";
+    if (inode != null) {
+      parentDir = inode.getFullPathName();
+    }
+    return (parentDir != null) ? parentDir : "";
   }
 
   /**

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RedundantEditLogInputStream.java

@@ -267,4 +267,11 @@ class RedundantEditLogInputStream extends EditLogInputStream {
       super(msg);
     }
   }
+
+  @Override
+  public void setMaxOpSize(int maxOpSize) {
+    for (EditLogInputStream elis : streams) {
+      elis.setMaxOpSize(maxOpSize);
+    }
+  }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/OfflineEditsXmlLoader.java

@@ -26,6 +26,7 @@ import java.util.Stack;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOp;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
@@ -176,7 +177,7 @@ class OfflineEditsXmlLoader
   
   @Override
   public void endElement (String uri, String name, String qName) {
-    String str = cbuf.toString().trim();
+    String str = XMLUtils.unmangleXmlString(cbuf.toString()).trim();
     cbuf = new StringBuffer();
     switch (state) {
     case EXPECT_EDITS_TAG:

+ 137 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/XMLUtils.java

@@ -46,6 +46,140 @@ public class XMLUtils {
     }
   }
   
+  /**
+   * Exception that reflects a string that cannot be unmangled.
+   */
+  public static class UnmanglingError extends RuntimeException {
+    private static final long serialVersionUID = 1L;
+    
+    public UnmanglingError(String str, Exception e) {
+      super(str, e);
+    }
+    
+    public UnmanglingError(String str) {
+      super(str);
+    }
+  }
+  
+
+  /**
+   * Given a code point, determine if it should be mangled before being
+   * represented in an XML document.
+   * 
+   * Any code point that isn't valid in XML must be mangled.
+   * See http://en.wikipedia.org/wiki/Valid_characters_in_XML for a
+   * quick reference, or the w3 standard for the authoritative reference.
+   * 
+   * @param cp      The code point
+   * @return        True if the code point should be mangled
+   */
+  private static boolean codePointMustBeMangled(int cp) {
+    if (cp < 0x20) {
+      return ((cp != 0x9) && (cp != 0xa) && (cp != 0xd));
+    } else if ((0xd7ff < cp) && (cp < 0xe000)) {
+      return true;
+    } else if ((cp == 0xfffe) || (cp == 0xffff)) {
+      return true;
+    } else if (cp == 0x5c) {
+      // we mangle backslash to simplify decoding... it's
+      // easier if backslashes always begin mangled sequences. 
+      return true;
+    }
+    return false;
+  }
+
+  private static int NUM_SLASH_POSITIONS = 4;
+
+  private static String mangleCodePoint(int cp) {
+    return String.format("\\%0" + NUM_SLASH_POSITIONS + "x;", cp);
+  }
+
+  /**
+   * Mangle a string so that it can be represented in an XML document.
+   * 
+   * There are three kinds of code points in XML:
+   * - Those that can be represented normally,
+   * - Those that have to be escaped (for example, & must be represented 
+   *     as &amp;)
+   * - Those that cannot be represented at all in XML.
+   *
+   * The built-in SAX functions will handle the first two types for us just
+   * fine.  However, sometimes we come across a code point of the third type.
+   * In this case, we have to mangle the string in order to represent it at
+   * all.  We also mangle backslash to avoid confusing a backslash in the
+   * string with part our escape sequence.
+   * 
+   * The encoding used here is as follows: an illegal code point is
+   * represented as '\ABCD;', where ABCD is the hexadecimal value of 
+   * the code point.
+   *
+   * @param str     The input string.
+   *
+   * @return        The mangled string.
+   */
+  public static String mangleXmlString(String str) {
+    final StringBuilder bld = new StringBuilder();
+    final int length = str.length();
+    for (int offset = 0; offset < length; ) {
+       final int cp = str.codePointAt(offset);
+       final int len = Character.charCount(cp);
+       if (codePointMustBeMangled(cp)) {
+         bld.append(mangleCodePoint(cp));
+       } else {
+         for (int i = 0; i < len; i++) {
+           bld.append(str.charAt(offset + i));
+         }
+       }
+       offset += len;
+    }
+    return bld.toString();
+  }
+
+  /**
+   * Demangle a string from an XML document.
+   * See {@link #mangleXmlString(String)} for a description of the mangling
+   * format.
+   *
+   * @param str    The string to be demangled.
+   * 
+   * @return       The unmangled string
+   * @throws       UnmanglingError if the input is malformed.
+   */
+  public static String unmangleXmlString(String str)
+        throws UnmanglingError {
+    int slashPosition = -1;
+    String escapedCp = "";
+    StringBuilder bld = new StringBuilder();
+    for (int i = 0; i < str.length(); i++) {
+      char ch = str.charAt(i);
+      if ((slashPosition >= 0) && (slashPosition < NUM_SLASH_POSITIONS)) {
+        escapedCp += ch;
+        ++slashPosition;
+      } else if (slashPosition == NUM_SLASH_POSITIONS) {
+        if (ch != ';') {
+          throw new UnmanglingError("unterminated code point escape: " +
+              "expected semicolon at end.");
+        }
+        try {
+          bld.appendCodePoint(Integer.parseInt(escapedCp, 16));
+        } catch (NumberFormatException e) {
+          throw new UnmanglingError("error parsing unmangling escape code", e);
+        }
+        escapedCp = "";
+        slashPosition = -1;
+      } else if (ch == '\\') {
+        slashPosition = 0;
+      } else {
+        bld.append(ch);
+      }
+    }
+    if (slashPosition != -1) {
+      throw new UnmanglingError("unterminated code point escape: string " +
+          "broke off in the middle");
+    }
+    return bld.toString();
+  }
+  
   /**
    * Add a SAX tag with a string inside.
    *
@@ -56,7 +190,7 @@ public class XMLUtils {
   public static void addSaxString(ContentHandler contentHandler,
       String tag, String val) throws SAXException {
     contentHandler.startElement("", "", tag, new AttributesImpl());
-    char c[] = val.toString().toCharArray();
+    char c[] = mangleXmlString(val).toCharArray();
     contentHandler.characters(c, 0, c.length);
     contentHandler.endElement("", "", tag);
   }
@@ -67,6 +201,8 @@ public class XMLUtils {
    */
   static public class Stanza {
     private TreeMap<String, LinkedList <Stanza > > subtrees;
+
+    /** The unmangled value of this stanza. */
     private String value;
     
     public Stanza() {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
@@ -244,7 +245,8 @@ public class JsonUtil {
     final long mTime = (Long) m.get("modificationTime");
     final long blockSize = (Long) m.get("blockSize");
     final short replication = (short) (long) (Long) m.get("replication");
-    final long fileId = (Long) m.get("fileId");
+    final long fileId = m.containsKey("fileId") ? (Long) m.get("fileId")
+        : INodeId.GRANDFATHER_INODE_ID;
     return new HdfsFileStatus(len, type == PathType.DIRECTORY, replication,
         blockSize, mTime, aTime, permission, owner, group,
         symlink, DFSUtil.string2Bytes(localName), fileId);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java

@@ -143,7 +143,7 @@ public class OfflineEditsViewerHelper {
       (DistributedFileSystem)cluster.getFileSystem();
     FileContext fc = FileContext.getFileContext(cluster.getURI(0), config);
     // OP_ADD 0, OP_SET_GENSTAMP 10
-    Path pathFileCreate = new Path("/file_create");
+    Path pathFileCreate = new Path("/file_create_u\1F431");
     FSDataOutputStream s = dfs.create(pathFileCreate);
     // OP_CLOSE 9
     s.close();

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

@@ -861,6 +861,11 @@ public class TestEditLog {
     public boolean isInProgress() {
       return true;
     }
+
+    @Override
+    public void setMaxOpSize(int maxOpSize) {
+      reader.setMaxOpSize(maxOpSize);
+    }
   }
 
   @Test

+ 52 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java

@@ -83,6 +83,7 @@ public class TestNameNodeRecovery {
       elfos.close();
       elfos = null;
       elfis = new EditLogFileInputStream(TEST_LOG_NAME);
+      elfis.setMaxOpSize(elts.getMaxOpSize());
       
       // reading through normally will get you an exception
       Set<Long> validTxIds = elts.getValidTxIds();
@@ -143,7 +144,7 @@ public class TestNameNodeRecovery {
   /**
    * A test scenario for the edit log
    */
-  private interface EditLogTestSetup {
+  private static abstract class EditLogTestSetup {
     /** 
      * Set up the edit log.
      */
@@ -162,6 +163,13 @@ public class TestNameNodeRecovery {
      * edit log.
      **/
     abstract public Set<Long> getValidTxIds();
+    
+    /**
+     * Return the maximum opcode size we will use for input.
+     */
+    public int getMaxOpSize() {
+      return DFSConfigKeys.DFS_NAMENODE_MAX_OP_SIZE_DEFAULT;
+    }
   }
   
   static void padEditLog(EditLogOutputStream elos, int paddingLength)
@@ -182,10 +190,10 @@ public class TestNameNodeRecovery {
   }
 
   static void addDeleteOpcode(EditLogOutputStream elos,
-        OpInstanceCache cache) throws IOException {
+        OpInstanceCache cache, long txId, String path) throws IOException {
     DeleteOp op = DeleteOp.getInstance(cache);
-    op.setTransactionId(0x0);
-    op.setPath("/foo");
+    op.setTransactionId(txId);
+    op.setPath(path);
     op.setTimestamp(0);
     elos.write(op);
   }
@@ -198,7 +206,7 @@ public class TestNameNodeRecovery {
    * able to handle any amount of padding (including no padding) without
    * throwing an exception.
    */
-  private static class EltsTestEmptyLog implements EditLogTestSetup {
+  private static class EltsTestEmptyLog extends EditLogTestSetup {
     private int paddingLength;
 
     public EltsTestEmptyLog(int paddingLength) {
@@ -242,6 +250,42 @@ public class TestNameNodeRecovery {
         3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
   }
 
+  /**
+   * Test using a non-default maximum opcode length.
+   */
+  private static class EltsTestNonDefaultMaxOpSize extends EditLogTestSetup {
+    public EltsTestNonDefaultMaxOpSize() {
+    }
+
+    @Override
+    public void addTransactionsToLog(EditLogOutputStream elos,
+        OpInstanceCache cache) throws IOException {
+      addDeleteOpcode(elos, cache, 0, "/foo");
+      addDeleteOpcode(elos, cache, 1,
+       "/supercalifragalisticexpialadocius.supercalifragalisticexpialadocius");
+    }
+
+    @Override
+    public long getLastValidTxId() {
+      return 0;
+    }
+
+    @Override
+    public Set<Long> getValidTxIds() {
+      return Sets.newHashSet(0L);
+    } 
+    
+    public int getMaxOpSize() {
+      return 30;
+    }
+  }
+
+  /** Test an empty edit log with extra-long padding */
+  @Test(timeout=180000)
+  public void testNonDefaultMaxOpSize() throws IOException {
+    runEditLogTest(new EltsTestNonDefaultMaxOpSize());
+  }
+
   /**
    * Test the scenario where an edit log contains some padding (0xff) bytes
    * followed by valid opcode data.
@@ -249,7 +293,7 @@ public class TestNameNodeRecovery {
    * These edit logs are corrupt, but all the opcodes should be recoverable
    * with recovery mode.
    */
-  private static class EltsTestOpcodesAfterPadding implements EditLogTestSetup {
+  private static class EltsTestOpcodesAfterPadding extends EditLogTestSetup {
     private int paddingLength;
 
     public EltsTestOpcodesAfterPadding(int paddingLength) {
@@ -260,7 +304,7 @@ public class TestNameNodeRecovery {
     public void addTransactionsToLog(EditLogOutputStream elos,
         OpInstanceCache cache) throws IOException {
       padEditLog(elos, paddingLength);
-      addDeleteOpcode(elos, cache);
+      addDeleteOpcode(elos, cache, 0, "/foo");
     }
 
     @Override
@@ -286,7 +330,7 @@ public class TestNameNodeRecovery {
         3 * EditLogFileOutputStream.MIN_PREALLOCATION_LENGTH));
   }
 
-  private static class EltsTestGarbageInEditLog implements EditLogTestSetup {
+  private static class EltsTestGarbageInEditLog extends EditLogTestSetup {
     final private long BAD_TXID = 4;
     final private long MAX_TXID = 10;
     

+ 19 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java

@@ -123,7 +123,25 @@ public class TestNameNodeMetrics {
     stm.read(buffer,0,4);
     stm.close();
   }
-  
+
+  /**
+   * Test that capacity metrics are exported and pass
+   * basic sanity tests.
+   */
+  @Test (timeout = 1800)
+  public void testCapacityMetrics() throws Exception {
+    MetricsRecordBuilder rb = getMetrics(NS_METRICS);
+    long capacityTotal = MetricsAsserts.getLongGauge("CapacityTotal", rb);
+    assert(capacityTotal != 0);
+    long capacityUsed = MetricsAsserts.getLongGauge("CapacityUsed", rb);
+    long capacityRemaining =
+        MetricsAsserts.getLongGauge("CapacityRemaining", rb);
+    long capacityUsedNonDFS =
+        MetricsAsserts.getLongGauge("CapacityUsedNonDFS", rb);
+    assert(capacityUsed + capacityRemaining + capacityUsedNonDFS ==
+        capacityTotal);
+  }
+
   /** Test metrics indicating the number of stale DataNodes */
   @Test
   public void testStaleNodes() throws Exception {

+ 70 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestXMLUtils.java

@@ -0,0 +1,70 @@
+/**
+ * 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.hdfs.util;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hdfs.util.XMLUtils.UnmanglingError;
+import org.junit.Test;
+
+public class TestXMLUtils {
+  private static void testRoundTrip(String str, String expectedMangled) {
+    String mangled = XMLUtils.mangleXmlString(str);
+    Assert.assertEquals(mangled, expectedMangled);
+    String unmangled = XMLUtils.unmangleXmlString(mangled);
+    Assert.assertEquals(unmangled, str);
+  }
+
+  @Test
+  public void testMangleEmptyString() throws Exception {
+    testRoundTrip("", "");
+  }
+
+  @Test
+  public void testMangleVanillaString() throws Exception {
+    testRoundTrip("abcdef", "abcdef");
+  }
+
+  @Test
+  public void testMangleStringWithBackSlash() throws Exception {
+    testRoundTrip("a\\bcdef", "a\\005c;bcdef");
+    testRoundTrip("\\\\", "\\005c;\\005c;");
+  }  
+
+  @Test
+  public void testMangleStringWithForbiddenCodePoint() throws Exception {
+    testRoundTrip("a\u0001bcdef", "a\\0001;bcdef");
+    testRoundTrip("a\u0002\ud800bcdef", "a\\0002;\\d800;bcdef");
+  }
+
+  @Test
+  public void testInvalidSequence() throws Exception {
+    try {
+      XMLUtils.unmangleXmlString("\\000g;foo");
+      Assert.fail("expected an unmangling error");
+    } catch (UnmanglingError e) {
+      // pass through
+    }
+    try {
+      XMLUtils.unmangleXmlString("\\0");
+      Assert.fail("expected an unmangling error");
+    } catch (UnmanglingError e) {
+      // pass through
+    }
+  }
+}

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

@@ -163,6 +163,12 @@ Release 2.0.4-beta - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5033. mapred shell script should respect usage flags
+    (--help -help -h). (Andrew Wang via atm)
+
+    MAPREDUCE-4892. Modify CombineFileInputFormat to not skew input slits'
+    allocation on small clusters. (Bikas Saha via vinodkv)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -177,6 +183,24 @@ Release 2.0.4-beta - UNRELEASED
 
     MAPREDUCE-4994. Addendum fixing testcases failures. (sandyr via tucu)
 
+    MAPREDUCE-4846. Some JobQueueInfo methods are public in MR1 but protected
+    in MR2. (Sandy Ryza via tomwhite)
+
+    MAPREDUCE-5013. mapred.JobStatus compatibility: MR2 missing constructors
+    from MR1. (Sandy Ryza via tomwhite)
+
+    MAPREDUCE-4951. Container preemption interpreted as task failure.
+    (Sandy Ryza via tomwhite)
+
+    MAPREDUCE-5008. Merger progress miscounts with respect to EOF_MARKER.
+    (Sandy Ryza via tomwhite)
+
+    MAPREDUCE-4693. History server should include counters for failed tasks.
+    (Xuan Gong via sseth)
+
+    MAPREDUCE-4896. mapred queue -info spits out ugly exception when queue does 
+    not exist. (sandyr via tucu)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -726,6 +750,9 @@ Release 0.23.7 - UNRELEASED
     MAPREDUCE-5009. Killing the Task Attempt slated for commit does not clear
     the value from the Task commitAttempt member (Robert Parker via jeagles)
 
+    MAPREDUCE-4871. AM uses mapreduce.jobtracker.split.metainfo.maxsize but
+    mapred-default has mapreduce.job.split.metainfo.maxsize (Jason Lowe via
+    jeagles)
 
 Release 0.23.6 - UNRELEASED
 

+ 8 - 0
hadoop-mapreduce-project/bin/mapred

@@ -50,6 +50,14 @@ fi
 COMMAND=$1
 shift
 
+case $COMMAND in
+  # usage flags
+  --help|-help|-h)
+    print_usage
+    exit
+    ;;
+esac
+
 if [ "$COMMAND" = "job" ] ; then
   CLASS=org.apache.hadoop.mapred.JobClient
 elif [ "$COMMAND" = "queue" ] ; then

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -238,7 +238,6 @@ public abstract class TaskAttemptImpl implements
          TaskAttemptStateInternal.FAIL_CONTAINER_CLEANUP,
          TaskAttemptEventType.TA_CONTAINER_COMPLETED,
          CLEANUP_CONTAINER_TRANSITION)
-      // ^ If RM kills the container due to expiry, preemption etc. 
      .addTransition(TaskAttemptStateInternal.ASSIGNED, 
          TaskAttemptStateInternal.KILL_CONTAINER_CLEANUP,
          TaskAttemptEventType.TA_KILL, CLEANUP_CONTAINER_TRANSITION)
@@ -1184,7 +1183,8 @@ public abstract class TaskAttemptImpl implements
             taskAttempt.nodeRackName == null ? "UNKNOWN" 
                 : taskAttempt.nodeRackName,
             StringUtils.join(
-                LINE_SEPARATOR, taskAttempt.getDiagnostics()), taskAttempt
+                LINE_SEPARATOR, taskAttempt.getDiagnostics()),
+                taskAttempt.getCounters(), taskAttempt
                 .getProgressSplitBlock().burst());
     return tauce;
   }

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskImpl.java

@@ -730,7 +730,8 @@ public abstract class TaskImpl implements Task, EventHandler<TaskEvent> {
         TypeConverter.fromYarn(task.getType()),
         errorSb.toString(),
         taskState.toString(),
-        taId == null ? null : TypeConverter.fromYarn(taId));
+        taId == null ? null : TypeConverter.fromYarn(taId),
+        task.getCounters());
     return taskFailedEvent;
   }
   

+ 18 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java

@@ -67,9 +67,12 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.RackResolver;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * Allocates the container from the ResourceManager scheduler.
  */
@@ -606,8 +609,8 @@ public class RMContainerAllocator extends RMContainerRequestor
         assignedRequests.remove(attemptID);
         
         // send the container completed event to Task attempt
-        eventHandler.handle(new TaskAttemptEvent(attemptID,
-            TaskAttemptEventType.TA_CONTAINER_COMPLETED));
+        eventHandler.handle(createContainerFinishedEvent(cont, attemptID));
+        
         // Send the diagnostics
         String diagnostics = StringInterner.weakIntern(cont.getDiagnostics());
         eventHandler.handle(new TaskAttemptDiagnosticsUpdateEvent(attemptID,
@@ -617,6 +620,19 @@ public class RMContainerAllocator extends RMContainerRequestor
     return newContainers;
   }
   
+  @VisibleForTesting
+  public TaskAttemptEvent createContainerFinishedEvent(ContainerStatus cont,
+      TaskAttemptId attemptID) {
+    if (cont.getExitStatus() == YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS) {
+      // killed by framework
+      return new TaskAttemptEvent(attemptID,
+          TaskAttemptEventType.TA_KILL);
+    } else {
+      return new TaskAttemptEvent(attemptID,
+          TaskAttemptEventType.TA_CONTAINER_COMPLETED);
+    }
+  }
+  
   @SuppressWarnings("unchecked")
   private void handleUpdatedNodes(AMResponse response) {
     // send event to the job about on updated nodes

+ 27 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java

@@ -83,6 +83,7 @@ import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.Event;
@@ -1645,6 +1646,32 @@ public class TestRMContainerAllocator {
     Assert.assertTrue(callbackCalled.get());
   }
 
+  @Test
+  public void testCompletedContainerEvent() {
+    RMContainerAllocator allocator = new RMContainerAllocator(
+        mock(ClientService.class), mock(AppContext.class));
+    
+    TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(
+        MRBuilderUtils.newTaskId(
+            MRBuilderUtils.newJobId(1, 1, 1), 1, TaskType.MAP), 1);
+    ContainerId containerId = BuilderUtils.newContainerId(1, 1, 1, 1);
+    ContainerStatus status = BuilderUtils.newContainerStatus(
+        containerId, ContainerState.RUNNING, "", 0);
+
+    ContainerStatus abortedStatus = BuilderUtils.newContainerStatus(
+        containerId, ContainerState.RUNNING, "",
+        YarnConfiguration.ABORTED_CONTAINER_EXIT_STATUS);
+    
+    TaskAttemptEvent event = allocator.createContainerFinishedEvent(status,
+        attemptId);
+    Assert.assertEquals(TaskAttemptEventType.TA_CONTAINER_COMPLETED,
+        event.getType());
+    
+    TaskAttemptEvent abortedEvent = allocator.createContainerFinishedEvent(
+        abortedStatus, attemptId);
+    Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType());
+  }
+  
   public static void main(String[] args) throws Exception {
     TestRMContainerAllocator t = new TestRMContainerAllocator();
     t.testSimple();

+ 3 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/avro/Events.avpr

@@ -212,6 +212,7 @@
           {"name": "rackname", "type": "string"},
           {"name": "status", "type": "string"},
           {"name": "error", "type": "string"},
+          {"name": "counters", "type": "JhCounters"},
           {"name": "clockSplits", "type": { "type": "array", "items": "int"}},
           {"name": "cpuUsages", "type": { "type": "array", "items": "int"}},
           {"name": "vMemKbytes", "type": { "type": "array", "items": "int"}},
@@ -226,7 +227,8 @@
           {"name": "finishTime", "type": "long"},
           {"name": "error", "type": "string"},
           {"name": "failedDueToAttempt", "type": ["null", "string"] },
-          {"name": "status", "type": "string"}
+          {"name": "status", "type": "string"},
+          {"name": "counters", "type": "JhCounters"}
       ]
      },
 

+ 4 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobConf.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.util.ClassUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.Tool;
 import org.apache.log4j.Level;
@@ -453,7 +454,7 @@ public class JobConf extends Configuration {
    * @param cls the example class.
    */
   public void setJarByClass(Class cls) {
-    String jar = findContainingJar(cls);
+    String jar = ClassUtil.findContainingJar(cls);
     if (jar != null) {
       setJar(jar);
     }   
@@ -1811,7 +1812,7 @@ public class JobConf extends Configuration {
     return 
     (int)(Math.ceil((float)getMemoryForReduceTask() / (float)slotSizePerReduce));
   }
-  
+
   /** 
    * Find a jar that contains a class of the same name, if any.
    * It will return a jar file, even if that is not the first thing
@@ -1822,35 +1823,9 @@ public class JobConf extends Configuration {
    * @throws IOException
    */
   public static String findContainingJar(Class my_class) {
-    ClassLoader loader = my_class.getClassLoader();
-    String class_file = my_class.getName().replaceAll("\\.", "/") + ".class";
-    try {
-      for(Enumeration itr = loader.getResources(class_file);
-          itr.hasMoreElements();) {
-        URL url = (URL) itr.nextElement();
-        if ("jar".equals(url.getProtocol())) {
-          String toReturn = url.getPath();
-          if (toReturn.startsWith("file:")) {
-            toReturn = toReturn.substring("file:".length());
-          }
-          // URLDecoder is a misnamed class, since it actually decodes
-          // x-www-form-urlencoded MIME type rather than actual
-          // URL encoding (which the file path has). Therefore it would
-          // decode +s to ' 's which is incorrect (spaces are actually
-          // either unencoded or encoded as "%20"). Replace +s first, so
-          // that they are kept sacred during the decoding process.
-          toReturn = toReturn.replaceAll("\\+", "%2B");
-          toReturn = URLDecoder.decode(toReturn, "UTF-8");
-          return toReturn.replaceAll("!.*$", "");
-        }
-      }
-    } catch (IOException e) {
-      throw new RuntimeException(e);
-    }
-    return null;
+    return ClassUtil.findContainingJar(my_class);
   }
 
-
   /**
    * Get the memory required to run a task of this job, in bytes. See
    * {@link #MAPRED_TASK_MAXVMEM_PROPERTY}

+ 17 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueInfo.java

@@ -67,7 +67,8 @@ public class JobQueueInfo extends QueueInfo {
    * 
    * @param queueName Name of the job queue.
    */
-  protected void setQueueName(String queueName) {
+  @InterfaceAudience.Private
+  public void setQueueName(String queueName) {
     super.setQueueName(queueName);
   }
 
@@ -76,7 +77,8 @@ public class JobQueueInfo extends QueueInfo {
    * 
    * @param schedulingInfo
    */
-  protected void setSchedulingInfo(String schedulingInfo) {
+  @InterfaceAudience.Private
+  public void setSchedulingInfo(String schedulingInfo) {
     super.setSchedulingInfo(schedulingInfo);
   }
 
@@ -84,15 +86,21 @@ public class JobQueueInfo extends QueueInfo {
    * Set the state of the queue
    * @param state state of the queue.
    */
-  protected void setQueueState(String state) {
+  @InterfaceAudience.Private
+  public void setQueueState(String state) {
     super.setState(QueueState.getState(state));
   }
   
-  String getQueueState() {
+  /**
+   * Use getState() instead
+   */
+  @Deprecated
+  public String getQueueState() {
     return super.getState().toString();
   }
   
-  protected void setChildren(List<JobQueueInfo> children) {
+  @InterfaceAudience.Private
+  public void setChildren(List<JobQueueInfo> children) {
     List<QueueInfo> list = new ArrayList<QueueInfo>();
     for (JobQueueInfo q : children) {
       list.add(q);
@@ -108,7 +116,8 @@ public class JobQueueInfo extends QueueInfo {
     return list;
   }
 
-  protected void setProperties(Properties props) {
+  @InterfaceAudience.Private
+  public void setProperties(Properties props) {
     super.setProperties(props);
   }
 
@@ -141,7 +150,8 @@ public class JobQueueInfo extends QueueInfo {
     setChildren(children);
   }
 
-  protected void setJobStatuses(org.apache.hadoop.mapreduce.JobStatus[] stats) {
+  @InterfaceAudience.Private
+  public void setJobStatuses(org.apache.hadoop.mapreduce.JobStatus[] stats) {
     super.setJobStatuses(stats);
   }
 

+ 53 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java

@@ -77,6 +77,59 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
    */
   public JobStatus() {
   }
+  
+  @Deprecated
+  public JobStatus(JobID jobid, float mapProgress, float reduceProgress,
+      float cleanupProgress, int runState) {
+    this(jobid, mapProgress, reduceProgress, cleanupProgress, runState, null,
+        null, null, null);
+  }
+
+  /**
+   * Create a job status object for a given jobid.
+   * @param jobid The jobid of the job
+   * @param mapProgress The progress made on the maps
+   * @param reduceProgress The progress made on the reduces
+   * @param runState The current state of the job
+   */
+  @Deprecated
+  public JobStatus(JobID jobid, float mapProgress, float reduceProgress,
+      int runState) {
+    this (jobid, mapProgress, reduceProgress, runState, null, null, null, null);
+  }
+
+  /**
+   * Create a job status object for a given jobid.
+   * @param jobid The jobid of the job
+   * @param mapProgress The progress made on the maps
+   * @param reduceProgress The progress made on the reduces
+   * @param runState The current state of the job
+   * @param jp Priority of the job.
+   */
+  @Deprecated
+  public JobStatus(JobID jobid, float mapProgress, float reduceProgress,
+      float cleanupProgress, int runState, JobPriority jp) {
+    this(jobid, mapProgress, reduceProgress, cleanupProgress, runState, jp,
+        null, null, null, null);
+  }
+
+  /**
+   * Create a job status object for a given jobid.
+   * @param jobid The jobid of the job
+   * @param setupProgress The progress made on the setup
+   * @param mapProgress The progress made on the maps
+   * @param reduceProgress The progress made on the reduces
+   * @param cleanupProgress The progress made on the cleanup
+   * @param runState The current state of the job
+   * @param jp Priority of the job.
+   */
+  @Deprecated
+  public JobStatus(JobID jobid, float setupProgress, float mapProgress,
+      float reduceProgress, float cleanupProgress, 
+      int runState, JobPriority jp) {
+    this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
+        runState, jp, null, null, null, null);
+  }
 
   /**
    * Create a job status object for a given jobid.

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

@@ -449,7 +449,7 @@ class QueueConfigurationParser {
     q.appendChild(propsElement);
 
     // Queue-state
-    String queueState = jqi.getQueueState();
+    String queueState = jqi.getState().getStateName();
     if (queueState != null
         && !queueState.equals(QueueState.UNDEFINED.getStateName())) {
       Element qStateElement = document.createElement(STATE_TAG);

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java

@@ -63,6 +63,9 @@ public interface MRJobConfig {
 
   public static final String SPLIT_FILE = "mapreduce.job.splitfile";
 
+  public static final String SPLIT_METAINFO_MAXSIZE = "mapreduce.job.split.metainfo.maxsize";
+  public static final long DEFAULT_SPLIT_METAINFO_MAXSIZE = 10000000L;
+
   public static final String NUM_MAPS = "mapreduce.job.maps";
 
   public static final String MAX_TASK_FAILURES_PER_TRACKER = "mapreduce.job.maxtaskfailures.per.tracker";

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java

@@ -295,6 +295,7 @@ public class JobHistoryParser implements HistoryEventHandler {
     attemptInfo.shuffleFinishTime = event.getFinishTime();
     attemptInfo.sortFinishTime = event.getFinishTime();
     attemptInfo.mapFinishTime = event.getFinishTime();
+    attemptInfo.counters = event.getCounters();
     if(TaskStatus.State.SUCCEEDED.toString().equals(taskInfo.status))
     {
       //this is a successful task
@@ -347,6 +348,7 @@ public class JobHistoryParser implements HistoryEventHandler {
     taskInfo.finishTime = event.getFinishTime();
     taskInfo.error = StringInterner.weakIntern(event.getError());
     taskInfo.failedDueToAttemptId = event.getFailedAttemptID();
+    taskInfo.counters = event.getCounters();
     info.errorInfo = "Task " + taskInfo.taskId +" failed " +
     taskInfo.attemptsMap.size() + " times ";
   }

+ 123 - 44
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.jobhistory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.TaskStatus;
+import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
@@ -36,8 +37,24 @@ import org.apache.avro.util.Utf8;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
-  private TaskAttemptUnsuccessfulCompletion datum =
-    new TaskAttemptUnsuccessfulCompletion();
+
+  private TaskAttemptUnsuccessfulCompletion datum = null;
+
+  private TaskAttemptID attemptId;
+  private TaskType taskType;
+  private String status;
+  private long finishTime;
+  private String hostname;
+  private int port;
+  private String rackName;
+  private String error;
+  private Counters counters;
+  int[][] allSplits;
+  int[] clockSplits;
+  int[] cpuUsages;
+  int[] vMemKbytes;
+  int[] physMemKbytes;
+  private static final Counters EMPTY_COUNTERS = new Counters();
 
   /** 
    * Create an event to record the unsuccessful completion of attempts
@@ -49,6 +66,7 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
    * @param port rpc port for for the tracker
    * @param rackName Name of the rack where the attempt executed
    * @param error Error string
+   * @param counters Counters for the attempt
    * @param allSplits the "splits", or a pixelated graph of various
    *        measurable worker node state variables against progress.
    *        Currently there are four; wallclock time, CPU time,
@@ -58,31 +76,25 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
        (TaskAttemptID id, TaskType taskType,
         String status, long finishTime,
         String hostname, int port, String rackName,
-        String error, int[][] allSplits) {
-    datum.taskid = new Utf8(id.getTaskID().toString());
-    datum.taskType = new Utf8(taskType.name());
-    datum.attemptId = new Utf8(id.toString());
-    datum.finishTime = finishTime;
-    datum.hostname = new Utf8(hostname);
-    if (rackName != null) {
-      datum.rackname = new Utf8(rackName);
-    }
-    datum.port = port;
-    datum.error = new Utf8(error);
-    datum.status = new Utf8(status);
-
-    datum.clockSplits 
-      = AvroArrayUtils.toAvro
-           (ProgressSplitsBlock.arrayGetWallclockTime(allSplits));
-    datum.cpuUsages 
-      = AvroArrayUtils.toAvro
-           (ProgressSplitsBlock.arrayGetCPUTime(allSplits));
-    datum.vMemKbytes 
-      = AvroArrayUtils.toAvro
-           (ProgressSplitsBlock.arrayGetVMemKbytes(allSplits));
-    datum.physMemKbytes 
-      = AvroArrayUtils.toAvro
-           (ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits));
+        String error, Counters counters, int[][] allSplits) {
+    this.attemptId = id;
+    this.taskType = taskType;
+    this.status = status;
+    this.finishTime = finishTime;
+    this.hostname = hostname;
+    this.port = port;
+    this.rackName = rackName;
+    this.error = error;
+    this.counters = counters;
+    this.allSplits = allSplits;
+    this.clockSplits =
+        ProgressSplitsBlock.arrayGetWallclockTime(allSplits);
+    this.cpuUsages =
+        ProgressSplitsBlock.arrayGetCPUTime(allSplits);
+    this.vMemKbytes =
+        ProgressSplitsBlock.arrayGetVMemKbytes(allSplits);
+    this.physMemKbytes =
+        ProgressSplitsBlock.arrayGetPhysMemKbytes(allSplits);
   }
 
   /** 
@@ -103,42 +115,109 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
        (TaskAttemptID id, TaskType taskType,
         String status, long finishTime, 
         String hostname, String error) {
-    this(id, taskType, status, finishTime, hostname, -1, "", error, null);
+    this(id, taskType, status, finishTime, hostname, -1, "",
+        error, EMPTY_COUNTERS, null);
+  }
+  
+  public TaskAttemptUnsuccessfulCompletionEvent
+      (TaskAttemptID id, TaskType taskType,
+       String status, long finishTime,
+       String hostname, int port, String rackName,
+       String error, int[][] allSplits) {
+    this(id, taskType, status, finishTime, hostname, port,
+        rackName, error, EMPTY_COUNTERS, null);
   }
 
   TaskAttemptUnsuccessfulCompletionEvent() {}
 
-  public Object getDatum() { return datum; }
-  public void setDatum(Object datum) {
-    this.datum = (TaskAttemptUnsuccessfulCompletion)datum;
+  public Object getDatum() {
+    if(datum == null) {
+      datum = new TaskAttemptUnsuccessfulCompletion();
+      datum.taskid = new Utf8(attemptId.getTaskID().toString());
+      datum.taskType = new Utf8(taskType.name());
+      datum.attemptId = new Utf8(attemptId.toString());
+      datum.finishTime = finishTime;
+      datum.hostname = new Utf8(hostname);
+      if (rackName != null) {
+        datum.rackname = new Utf8(rackName);
+      }
+      datum.port = port;
+      datum.error = new Utf8(error);
+      datum.status = new Utf8(status);
+
+      datum.counters = EventWriter.toAvro(counters);
+
+      datum.clockSplits = AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetWallclockTime(allSplits));
+      datum.cpuUsages = AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetCPUTime(allSplits));
+      datum.vMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetVMemKbytes(allSplits));
+      datum.physMemKbytes = AvroArrayUtils.toAvro(ProgressSplitsBlock
+          .arrayGetPhysMemKbytes(allSplits));
+    }
+    return datum;
+  }
+  
+  
+  
+  public void setDatum(Object odatum) {
+    this.datum =
+        (TaskAttemptUnsuccessfulCompletion)odatum;
+    this.attemptId =
+        TaskAttemptID.forName(datum.attemptId.toString());
+    this.taskType =
+        TaskType.valueOf(datum.taskType.toString());
+    this.finishTime = datum.finishTime;
+    this.hostname = datum.hostname.toString();
+    this.rackName = datum.rackname.toString();
+    this.port = datum.port;
+    this.status = datum.status.toString();
+    this.error = datum.error.toString();
+    this.counters =
+        EventReader.fromAvro(datum.counters);
+    this.clockSplits =
+        AvroArrayUtils.fromAvro(datum.clockSplits);
+    this.cpuUsages =
+        AvroArrayUtils.fromAvro(datum.cpuUsages);
+    this.vMemKbytes =
+        AvroArrayUtils.fromAvro(datum.vMemKbytes);
+    this.physMemKbytes =
+        AvroArrayUtils.fromAvro(datum.physMemKbytes);
   }
 
   /** Get the task id */
-  public TaskID getTaskId() { return TaskID.forName(datum.taskid.toString()); }
+  public TaskID getTaskId() {
+    return attemptId.getTaskID();
+  }
   /** Get the task type */
   public TaskType getTaskType() {
-    return TaskType.valueOf(datum.taskType.toString());
+    return TaskType.valueOf(taskType.toString());
   }
   /** Get the attempt id */
   public TaskAttemptID getTaskAttemptId() {
-    return TaskAttemptID.forName(datum.attemptId.toString());
+    return attemptId;
   }
   /** Get the finish time */
-  public long getFinishTime() { return datum.finishTime; }
+  public long getFinishTime() { return finishTime; }
   /** Get the name of the host where the attempt executed */
-  public String getHostname() { return datum.hostname.toString(); }
+  public String getHostname() { return hostname; }
   /** Get the rpc port for the host where the attempt executed */
-  public int getPort() { return datum.port; }
+  public int getPort() { return port; }
   
   /** Get the rack name of the node where the attempt ran */
   public String getRackName() {
-    return datum.rackname == null ? null : datum.rackname.toString();
+    return rackName == null ? null : rackName.toString();
   }
   
   /** Get the error string */
-  public String getError() { return datum.error.toString(); }
+  public String getError() { return error.toString(); }
   /** Get the task status */
-  public String getTaskStatus() { return datum.status.toString(); }
+  public String getTaskStatus() {
+    return status.toString();
+  }
+  /** Get the counters */
+  Counters getCounters() { return counters; }
   /** Get the event type */
   public EventType getEventType() {
     // Note that the task type can be setup/map/reduce/cleanup but the 
@@ -157,16 +236,16 @@ public class TaskAttemptUnsuccessfulCompletionEvent implements HistoryEvent {
 
 
   public int[] getClockSplits() {
-    return AvroArrayUtils.fromAvro(datum.clockSplits);
+    return clockSplits;
   }
   public int[] getCpuUsages() {
-    return AvroArrayUtils.fromAvro(datum.cpuUsages);
+    return cpuUsages;
   }
   public int[] getVMemKbytes() {
-    return AvroArrayUtils.fromAvro(datum.vMemKbytes);
+    return vMemKbytes;
   }
   public int[] getPhysMemKbytes() {
-    return AvroArrayUtils.fromAvro(datum.physMemKbytes);
+    return physMemKbytes;
   }
 
 }

+ 75 - 24
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskFailedEvent.java

@@ -18,10 +18,9 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
@@ -35,7 +34,17 @@ import org.apache.avro.util.Utf8;
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
 public class TaskFailedEvent implements HistoryEvent {
-  private TaskFailed datum = new TaskFailed();
+  private TaskFailed datum = null;
+
+  private TaskAttemptID failedDueToAttempt;
+  private TaskID id;
+  private TaskType taskType;
+  private long finishTime;
+  private String status;
+  private String error;
+  private Counters counters;
+
+  private static final Counters EMPTY_COUNTERS = new Counters();
 
   /**
    * Create an event to record task failure
@@ -45,45 +54,87 @@ public class TaskFailedEvent implements HistoryEvent {
    * @param error Error String
    * @param status Status
    * @param failedDueToAttempt The attempt id due to which the task failed
+   * @param counters Counters for the task
    */
   public TaskFailedEvent(TaskID id, long finishTime, 
       TaskType taskType, String error, String status,
-      TaskAttemptID failedDueToAttempt) {
-    datum.taskid = new Utf8(id.toString());
-    datum.error = new Utf8(error);
-    datum.finishTime = finishTime;
-    datum.taskType = new Utf8(taskType.name());
-    datum.failedDueToAttempt = failedDueToAttempt == null
-      ? null
-      : new Utf8(failedDueToAttempt.toString());
-    datum.status = new Utf8(status);
+      TaskAttemptID failedDueToAttempt, Counters counters) {
+    this.id = id;
+    this.finishTime = finishTime;
+    this.taskType = taskType;
+    this.error = error;
+    this.status = status;
+    this.failedDueToAttempt = failedDueToAttempt;
+    this.counters = counters;
   }
 
+  public TaskFailedEvent(TaskID id, long finishTime, 
+	      TaskType taskType, String error, String status,
+	      TaskAttemptID failedDueToAttempt) {
+    this(id, finishTime, taskType, error, status,
+        failedDueToAttempt, EMPTY_COUNTERS);
+  }
+  
   TaskFailedEvent() {}
 
-  public Object getDatum() { return datum; }
-  public void setDatum(Object datum) { this.datum = (TaskFailed)datum; }
+  public Object getDatum() {
+    if(datum == null) {
+      datum = new TaskFailed();
+      datum.taskid = new Utf8(id.toString());
+      datum.error = new Utf8(error);
+      datum.finishTime = finishTime;
+      datum.taskType = new Utf8(taskType.name());
+      datum.failedDueToAttempt =
+          failedDueToAttempt == null
+          ? null
+          : new Utf8(failedDueToAttempt.toString());
+      datum.status = new Utf8(status);
+      datum.counters = EventWriter.toAvro(counters);
+    }
+    return datum;
+  }
+  
+  public void setDatum(Object odatum) {
+    this.datum = (TaskFailed)odatum;
+    this.id =
+        TaskID.forName(datum.taskid.toString());
+    this.taskType =
+        TaskType.valueOf(datum.taskType.toString());
+    this.finishTime = datum.finishTime;
+    this.error = datum.error.toString();
+    this.failedDueToAttempt =
+        datum.failedDueToAttempt == null
+        ? null
+        : TaskAttemptID.forName(
+            datum.failedDueToAttempt.toString());
+    this.status = datum.status.toString();
+    this.counters =
+        EventReader.fromAvro(datum.counters);
+  }
 
   /** Get the task id */
-  public TaskID getTaskId() { return TaskID.forName(datum.taskid.toString()); }
+  public TaskID getTaskId() { return id; }
   /** Get the error string */
-  public String getError() { return datum.error.toString(); }
+  public String getError() { return error; }
   /** Get the finish time of the attempt */
-  public long getFinishTime() { return datum.finishTime; }
+  public long getFinishTime() {
+    return finishTime;
+  }
   /** Get the task type */
   public TaskType getTaskType() {
-    return TaskType.valueOf(datum.taskType.toString());
+    return taskType;
   }
   /** Get the attempt id due to which the task failed */
   public TaskAttemptID getFailedAttemptID() {
-    return datum.failedDueToAttempt == null
-      ? null
-      : TaskAttemptID.forName(datum.failedDueToAttempt.toString());
+    return failedDueToAttempt;
   }
   /** Get the task status */
-  public String getTaskStatus() { return datum.status.toString(); }
+  public String getTaskStatus() { return status; }
+  /** Get task counters */
+  public Counters getCounters() { return counters; }
   /** Get the event type */
-  public EventType getEventType() { return EventType.TASK_FAILED; }
+  public EventType getEventType() {
+    return EventType.TASK_FAILED;
+  }
 
-  
 }

+ 136 - 80
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/CombineFileInputFormat.java

@@ -49,6 +49,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NetworkTopology;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * An abstract {@link InputFormat} that returns {@link CombineFileSplit}'s in 
  * {@link InputFormat#getSplits(JobContext)} method. 
@@ -76,7 +78,7 @@ import org.apache.hadoop.net.NetworkTopology;
 @InterfaceStability.Stable
 public abstract class CombineFileInputFormat<K, V>
   extends FileInputFormat<K, V> {
-
+  
   public static final String SPLIT_MINSIZE_PERNODE = 
     "mapreduce.input.fileinputformat.split.minsize.per.node";
   public static final String SPLIT_MINSIZE_PERRACK = 
@@ -163,7 +165,6 @@ public abstract class CombineFileInputFormat<K, V>
   @Override
   public List<InputSplit> getSplits(JobContext job) 
     throws IOException {
-
     long minSizeNode = 0;
     long minSizeRack = 0;
     long maxSize = 0;
@@ -286,56 +287,100 @@ public abstract class CombineFileInputFormat<K, V>
                                  rackToNodes, maxSize);
       totLength += files[i].getLength();
     }
+    createSplits(nodeToBlocks, blockToNodes, rackToBlocks, totLength, 
+                 maxSize, minSizeNode, minSizeRack, splits);
+  }
 
+  @VisibleForTesting
+  void createSplits(HashMap<String, List<OneBlockInfo>> nodeToBlocks,
+                     HashMap<OneBlockInfo, String[]> blockToNodes,
+                     HashMap<String, List<OneBlockInfo>> rackToBlocks,
+                     long totLength,
+                     long maxSize,
+                     long minSizeNode,
+                     long minSizeRack,
+                     List<InputSplit> splits                     
+                    ) {
     ArrayList<OneBlockInfo> validBlocks = new ArrayList<OneBlockInfo>();
     Set<String> nodes = new HashSet<String>();
     long curSplitSize = 0;
+    
+    int numNodes = nodeToBlocks.size();
+    long totalLength = totLength;
+
+    while(true) {
+      // it is allowed for maxSize to be 0. Disable smoothing load for such cases
+      int avgSplitsPerNode = maxSize > 0 && numNodes > 0 ?
+                                        ((int) (totalLength/maxSize))/numNodes
+                                        : Integer.MAX_VALUE;
+      int maxSplitsByNodeOnly = (avgSplitsPerNode > 0) ? avgSplitsPerNode : 1;
+      numNodes = 0;
+
+      // process all nodes and create splits that are local to a node.
+      for (Iterator<Map.Entry<String, List<OneBlockInfo>>> iter = nodeToBlocks
+          .entrySet().iterator(); iter.hasNext();) {
+        Map.Entry<String, List<OneBlockInfo>> one = iter.next();
+        nodes.add(one.getKey());
+        List<OneBlockInfo> blocksInNode = one.getValue();
+
+        // for each block, copy it into validBlocks. Delete it from
+        // blockToNodes so that the same block does not appear in
+        // two different splits.
+        int splitsInNode = 0;
+        for (OneBlockInfo oneblock : blocksInNode) {
+          if (blockToNodes.containsKey(oneblock)) {
+            validBlocks.add(oneblock);
+            blockToNodes.remove(oneblock);
+            curSplitSize += oneblock.length;
 
-    // process all nodes and create splits that are local
-    // to a node. 
-    for (Iterator<Map.Entry<String, 
-         List<OneBlockInfo>>> iter = nodeToBlocks.entrySet().iterator(); 
-         iter.hasNext();) {
-
-      Map.Entry<String, List<OneBlockInfo>> one = iter.next();
-      nodes.add(one.getKey());
-      List<OneBlockInfo> blocksInNode = one.getValue();
-
-      // for each block, copy it into validBlocks. Delete it from 
-      // blockToNodes so that the same block does not appear in 
-      // two different splits.
-      for (OneBlockInfo oneblock : blocksInNode) {
-        if (blockToNodes.containsKey(oneblock)) {
-          validBlocks.add(oneblock);
-          blockToNodes.remove(oneblock);
-          curSplitSize += oneblock.length;
-
-          // if the accumulated split size exceeds the maximum, then 
-          // create this split.
-          if (maxSize != 0 && curSplitSize >= maxSize) {
-            // create an input split and add it to the splits array
-            addCreatedSplit(splits, nodes, validBlocks);
-            curSplitSize = 0;
-            validBlocks.clear();
+            // if the accumulated split size exceeds the maximum, then
+            // create this split.
+            if (maxSize != 0 && curSplitSize >= maxSize) {
+              // create an input split and add it to the splits array
+              addCreatedSplit(splits, nodes, validBlocks);
+              totalLength -= curSplitSize;
+              curSplitSize = 0;
+              validBlocks.clear();
+              splitsInNode++;
+              if (splitsInNode == maxSplitsByNodeOnly) {
+                // stop grouping on a node so as not to create
+                // disproportionately more splits on a node because it happens
+                // to have many blocks
+                // consider only these nodes in next round of grouping because
+                // they have leftover blocks that may need to be grouped
+                numNodes++;
+                break;
+              }
+            }
           }
         }
-      }
-      // if there were any blocks left over and their combined size is
-      // larger than minSplitNode, then combine them into one split.
-      // Otherwise add them back to the unprocessed pool. It is likely 
-      // that they will be combined with other blocks from the 
-      // same rack later on.
-      if (minSizeNode != 0 && curSplitSize >= minSizeNode) {
-        // create an input split and add it to the splits array
-        addCreatedSplit(splits, nodes, validBlocks);
-      } else {
-        for (OneBlockInfo oneblock : validBlocks) {
-          blockToNodes.put(oneblock, oneblock.hosts);
+        // if there were any blocks left over and their combined size is
+        // larger than minSplitNode, then combine them into one split.
+        // Otherwise add them back to the unprocessed pool. It is likely
+        // that they will be combined with other blocks from the
+        // same rack later on.
+        if (minSizeNode != 0 && curSplitSize >= minSizeNode
+            && splitsInNode == 0) {
+          // haven't created any split on this machine. so its ok to add a
+          // smaller
+          // one for parallelism. Otherwise group it in the rack for balanced
+          // size
+          // create an input split and add it to the splits array
+          addCreatedSplit(splits, nodes, validBlocks);
+          totalLength -= curSplitSize;
+        } else {
+          for (OneBlockInfo oneblock : validBlocks) {
+            blockToNodes.put(oneblock, oneblock.hosts);
+          }
         }
+        validBlocks.clear();
+        nodes.clear();
+        curSplitSize = 0;
+      }
+      
+      if(!(numNodes>0 && totalLength>0)) {
+        break;
       }
-      validBlocks.clear();
-      nodes.clear();
-      curSplitSize = 0;
     }
 
     // if blocks in a rack are below the specified minimum size, then keep them
@@ -458,7 +503,6 @@ public abstract class CombineFileInputFormat<K, V>
       offset[i] = validBlocks.get(i).offset;
       length[i] = validBlocks.get(i).length;
     }
-
      // add this split to the list that is returned
     CombineFileSplit thissplit = new CombineFileSplit(fl, offset, 
                                    length, locations.toArray(new String[0]));
@@ -474,7 +518,8 @@ public abstract class CombineFileInputFormat<K, V>
   /**
    * information about one file from the File System
    */
-  private static class OneFileInfo {
+  @VisibleForTesting
+  static class OneFileInfo {
     private long fileSize;               // size of the file
     private OneBlockInfo[] blocks;       // all blocks in this file
 
@@ -545,45 +590,55 @@ public abstract class CombineFileInputFormat<K, V>
           }
           blocks = blocksList.toArray(new OneBlockInfo[blocksList.size()]);
         }
+        
+        populateBlockInfo(blocks, rackToBlocks, blockToNodes, 
+                          nodeToBlocks, rackToNodes);
+      }
+    }
+    
+    @VisibleForTesting
+    static void populateBlockInfo(OneBlockInfo[] blocks,
+                          HashMap<String, List<OneBlockInfo>> rackToBlocks,
+                          HashMap<OneBlockInfo, String[]> blockToNodes,
+                          HashMap<String, List<OneBlockInfo>> nodeToBlocks,
+                          HashMap<String, Set<String>> rackToNodes) {
+      for (OneBlockInfo oneblock : blocks) {
+        // add this block to the block --> node locations map
+        blockToNodes.put(oneblock, oneblock.hosts);
+
+        // For blocks that do not have host/rack information,
+        // assign to default  rack.
+        String[] racks = null;
+        if (oneblock.hosts.length == 0) {
+          racks = new String[]{NetworkTopology.DEFAULT_RACK};
+        } else {
+          racks = oneblock.racks;
+        }
 
-        for (OneBlockInfo oneblock : blocks) {
-          // add this block to the block --> node locations map
-          blockToNodes.put(oneblock, oneblock.hosts);
-
-          // For blocks that do not have host/rack information,
-          // assign to default  rack.
-          String[] racks = null;
-          if (oneblock.hosts.length == 0) {
-            racks = new String[]{NetworkTopology.DEFAULT_RACK};
-          } else {
-            racks = oneblock.racks;
+        // add this block to the rack --> block map
+        for (int j = 0; j < racks.length; j++) {
+          String rack = racks[j];
+          List<OneBlockInfo> blklist = rackToBlocks.get(rack);
+          if (blklist == null) {
+            blklist = new ArrayList<OneBlockInfo>();
+            rackToBlocks.put(rack, blklist);
           }
-
-          // add this block to the rack --> block map
-          for (int j = 0; j < racks.length; j++) {
-            String rack = racks[j];
-            List<OneBlockInfo> blklist = rackToBlocks.get(rack);
-            if (blklist == null) {
-              blklist = new ArrayList<OneBlockInfo>();
-              rackToBlocks.put(rack, blklist);
-            }
-            blklist.add(oneblock);
-            if (!racks[j].equals(NetworkTopology.DEFAULT_RACK)) {
-              // Add this host to rackToNodes map
-              addHostToRack(rackToNodes, racks[j], oneblock.hosts[j]);
-            }
+          blklist.add(oneblock);
+          if (!racks[j].equals(NetworkTopology.DEFAULT_RACK)) {
+            // Add this host to rackToNodes map
+            addHostToRack(rackToNodes, racks[j], oneblock.hosts[j]);
           }
+        }
 
-          // add this block to the node --> block map
-          for (int j = 0; j < oneblock.hosts.length; j++) {
-            String node = oneblock.hosts[j];
-            List<OneBlockInfo> blklist = nodeToBlocks.get(node);
-            if (blklist == null) {
-              blklist = new ArrayList<OneBlockInfo>();
-              nodeToBlocks.put(node, blklist);
-            }
-            blklist.add(oneblock);
+        // add this block to the node --> block map
+        for (int j = 0; j < oneblock.hosts.length; j++) {
+          String node = oneblock.hosts[j];
+          List<OneBlockInfo> blklist = nodeToBlocks.get(node);
+          if (blklist == null) {
+            blklist = new ArrayList<OneBlockInfo>();
+            nodeToBlocks.put(node, blklist);
           }
+          blklist.add(oneblock);
         }
       }
     }
@@ -600,7 +655,8 @@ public abstract class CombineFileInputFormat<K, V>
   /**
    * information about one block from the File System
    */
-  private static class OneBlockInfo {
+  @VisibleForTesting
+  static class OneBlockInfo {
     Path onepath;                // name of this file
     long offset;                 // offset in file
     long length;                 // length of this block

+ 5 - 5
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/split/SplitMetaInfoReader.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.mapreduce.split;
 import java.io.IOException;
 import java.util.Arrays;
 
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -29,9 +31,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobSubmissionFiles;
-import org.apache.hadoop.mapreduce.server.jobtracker.JTConfig;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 
 /**
  * A utility that reads the split meta info and creates
@@ -44,8 +44,8 @@ public class SplitMetaInfoReader {
   public static JobSplit.TaskSplitMetaInfo[] readSplitMetaInfo(
       JobID jobId, FileSystem fs, Configuration conf, Path jobSubmitDir) 
   throws IOException {
-    long maxMetaInfoSize = conf.getLong(JTConfig.JT_MAX_JOB_SPLIT_METAINFO_SIZE, 
-        10000000L);
+    long maxMetaInfoSize = conf.getLong(MRJobConfig.SPLIT_METAINFO_MAXSIZE,
+        MRJobConfig.DEFAULT_SPLIT_METAINFO_MAXSIZE);
     Path metaSplitFile = JobSubmissionFiles.getJobSplitMetaFile(jobSubmitDir);
     String jobSplitFile = JobSubmissionFiles.getJobSplitFile(jobSubmitDir).toString();
     FileStatus fStatus = fs.getFileStatus(metaSplitFile);

+ 6 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManagerImpl.java

@@ -475,9 +475,9 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
           combineCollector.setWriter(writer);
           combineAndSpill(rIter, reduceCombineInputCounter);
         }
+        writer.close();
         compressAwarePath = new CompressAwarePath(outputPath,
             writer.getRawLength());
-        writer.close();
 
         LOG.info(reduceId +  
             " Merge of the " + noInMemorySegments +
@@ -552,9 +552,9 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
                             mergedMapOutputsCounter, null);
 
         Merger.writeFile(iter, writer, reporter, jobConf);
+        writer.close();
         compressAwarePath = new CompressAwarePath(outputPath,
             writer.getRawLength());
-        writer.close();
       } catch (IOException e) {
         localFS.delete(outputPath, true);
         throw e;
@@ -713,13 +713,15 @@ public class MergeManagerImpl<K, V> implements MergeManager<K, V> {
             keyClass, valueClass, memDiskSegments, numMemDiskSegments,
             tmpDir, comparator, reporter, spilledRecordsCounter, null, 
             mergePhase);
-        final Writer<K,V> writer = new Writer<K,V>(job, fs, outputPath,
+        Writer<K,V> writer = new Writer<K,V>(job, fs, outputPath,
             keyClass, valueClass, codec, null);
         try {
           Merger.writeFile(rIter, writer, reporter, job);
-          // add to list of final disk outputs.
+          writer.close();
           onDiskMapOutputs.add(new CompressAwarePath(outputPath,
               writer.getRawLength()));
+          writer = null;
+          // add to list of final disk outputs.
         } catch (IOException e) {
           if (null != outputPath) {
             try {

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ConfigUtil.java

@@ -521,6 +521,8 @@ public class ConfigUtil {
     });
     Configuration.addDeprecation("mapreduce.user.classpath.first",
       MRJobConfig.MAPREDUCE_JOB_USER_CLASSPATH_FIRST);
+    Configuration.addDeprecation(JTConfig.JT_MAX_JOB_SPLIT_METAINFO_SIZE,
+        MRJobConfig.SPLIT_METAINFO_MAXSIZE);
   }
 
   public static void main(String[] args) {

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

@@ -404,7 +404,7 @@ public class TestJobHistoryParsing {
     }
   }
   
-  @Test
+  @Test (timeout=5000)
   public void testCountersForFailedTask() throws Exception {
     LOG.info("STARTING testCountersForFailedTask");
     try {
@@ -455,6 +455,9 @@ public class TestJobHistoryParsing {
       CompletedTask ct = new CompletedTask(yarnTaskID, entry.getValue());
       Assert.assertNotNull("completed task report has null counters",
           ct.getReport().getCounters());
+      //Make sure all the completedTask has counters, and the counters are not empty
+      Assert.assertTrue(ct.getReport().getCounters()
+          .getAllCounterGroups().size() > 0);
     }
     } finally {
       LOG.info("FINISHED testCountersForFailedTask");

+ 3 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java

@@ -106,8 +106,9 @@ public class ResourceMgrDelegate extends YarnClientImpl {
 
   public QueueInfo getQueue(String queueName) throws IOException,
   InterruptedException {
-    return TypeConverter.fromYarn(
-        super.getQueueInfo(queueName), this.conf);
+    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo =
+        super.getQueueInfo(queueName);
+    return (queueInfo == null) ? null : TypeConverter.fromYarn(queueInfo, conf);
   }
 
   public QueueAclsInfo[] getQueueAclsForCurrentUser() throws IOException,

+ 2 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestJobConf.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.util.ClassUtil;
 
 
 import static org.junit.Assert.*;
@@ -79,7 +80,7 @@ public class TestJobConf {
     Class clazz = Class.forName(CLASSNAME, true, cl);
     assertNotNull(clazz);
 
-    String containingJar = JobConf.findContainingJar(clazz);
+    String containingJar = ClassUtil.findContainingJar(clazz);
     assertEquals(jar.getAbsolutePath(), containingJar);
   }
 }

+ 86 - 33
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -20,11 +20,14 @@ package org.apache.hadoop.mapreduce.lib.input;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.net.URI;
+import java.util.HashMap;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.Set;
 import java.util.zip.GZIPOutputStream;
 import java.util.concurrent.TimeoutException;
 
+import junit.framework.Assert;
 import junit.framework.TestCase;
 
 import org.apache.hadoop.fs.*;
@@ -42,9 +45,13 @@ import org.apache.hadoop.mapreduce.RecordReader;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
 import org.apache.hadoop.mapreduce.TaskType;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneBlockInfo;
+import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.OneFileInfo;
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
 import org.junit.Test;
 
+import com.google.common.collect.HashMultiset;
+
 public class TestCombineFileInputFormat extends TestCase {
 
   private static final String rack1[] = new String[] {
@@ -476,23 +483,23 @@ public class TestCombineFileInputFormat extends TestCase {
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
       assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
       fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file3.getName(), fileSplit.getPath(0).getName());
-      assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(0));
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
       assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
-      assertEquals(0, fileSplit.getOffset(1));
+      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+      assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
+      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
       fileSplit = (CombineFileSplit) splits.get(2);
       assertEquals(2, fileSplit.getNumPaths());
       assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
-      assertEquals(BLOCKSIZE, fileSplit.getOffset(0));
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(0, fileSplit.getOffset(0));
       assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
+      assertEquals(file3.getName(), fileSplit.getPath(1).getName());
       assertEquals(2 * BLOCKSIZE, fileSplit.getOffset(1));
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
+      assertEquals("host1.rack1.com", fileSplit.getLocations()[0]);
 
       // maximum split size is 3 blocks 
       inFormat = new DummyInputFormat();
@@ -504,7 +511,7 @@ public class TestCombineFileInputFormat extends TestCase {
       for (InputSplit split : splits) {
         System.out.println("File split(Test5): " + split);
       }
-      assertEquals(4, splits.size());
+      assertEquals(3, splits.size());
       fileSplit = (CombineFileSplit) splits.get(0);
       assertEquals(3, fileSplit.getNumPaths());
       assertEquals(1, fileSplit.getLocations().length);
@@ -519,32 +526,28 @@ public class TestCombineFileInputFormat extends TestCase {
       assertEquals(BLOCKSIZE, fileSplit.getLength(2));
       assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
       fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
       assertEquals(0, fileSplit.getOffset(0));
       assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
+      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
       assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
       assertEquals(file4.getName(), fileSplit.getPath(2).getName());
-      assertEquals( 2 * BLOCKSIZE, fileSplit.getOffset(2));
+      assertEquals(0, fileSplit.getOffset(2));
       assertEquals(BLOCKSIZE, fileSplit.getLength(2));
-      assertEquals("host3.rack3.com", fileSplit.getLocations()[0]);
+      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
       fileSplit = (CombineFileSplit) splits.get(2);
-      assertEquals(2, fileSplit.getNumPaths());
+      assertEquals(3, fileSplit.getNumPaths());
       assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
       assertEquals(0, fileSplit.getOffset(0));
       assertEquals(BLOCKSIZE, fileSplit.getLength(0));
-      assertEquals(file2.getName(), fileSplit.getPath(1).getName());
+      assertEquals(file4.getName(), fileSplit.getPath(1).getName());
       assertEquals(BLOCKSIZE, fileSplit.getOffset(1));
       assertEquals(BLOCKSIZE, fileSplit.getLength(1));
-      assertEquals("host2.rack2.com", fileSplit.getLocations()[0]);
-      fileSplit = (CombineFileSplit) splits.get(3);
-      assertEquals(1, fileSplit.getNumPaths());
-      assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
-      assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(BLOCKSIZE, fileSplit.getLength(0));
+      assertEquals(file4.getName(), fileSplit.getPath(2).getName());
+      assertEquals(2*BLOCKSIZE, fileSplit.getOffset(2));
+      assertEquals(BLOCKSIZE, fileSplit.getLength(2));
       assertEquals("host1.rack1.com", fileSplit.getLocations()[0]);
 
       // maximum split size is 4 blocks 
@@ -713,6 +716,56 @@ public class TestCombineFileInputFormat extends TestCase {
     DFSTestUtil.waitReplication(fileSys, name, replication);
   }
   
+  public void testNodeInputSplit() throws IOException, InterruptedException {
+    // Regression test for MAPREDUCE-4892. There are 2 nodes with all blocks on 
+    // both nodes. The grouping ensures that both nodes get splits instead of 
+    // just the first node
+    DummyInputFormat inFormat = new DummyInputFormat();
+    int numBlocks = 12;
+    long totLength = 0;
+    long blockSize = 100;
+    long maxSize = 200;
+    long minSizeNode = 50;
+    long minSizeRack = 50;
+    String[] locations = { "h1", "h2" };
+    String[] racks = new String[0];
+    Path path = new Path("hdfs://file");
+    
+    OneBlockInfo[] blocks = new OneBlockInfo[numBlocks];
+    for(int i=0; i<numBlocks; ++i) {
+      blocks[i] = new OneBlockInfo(path, i*blockSize, blockSize, locations, racks);
+      totLength += blockSize;
+    }
+    
+    List<InputSplit> splits = new ArrayList<InputSplit>();
+    HashMap<String, Set<String>> rackToNodes = 
+                              new HashMap<String, Set<String>>();
+    HashMap<String, List<OneBlockInfo>> rackToBlocks = 
+                              new HashMap<String, List<OneBlockInfo>>();
+    HashMap<OneBlockInfo, String[]> blockToNodes = 
+                              new HashMap<OneBlockInfo, String[]>();
+    HashMap<String, List<OneBlockInfo>> nodeToBlocks = 
+                              new HashMap<String, List<OneBlockInfo>>();
+    
+    OneFileInfo.populateBlockInfo(blocks, rackToBlocks, blockToNodes, 
+                             nodeToBlocks, rackToNodes);
+    
+    inFormat.createSplits(nodeToBlocks, blockToNodes, rackToBlocks, totLength,  
+                          maxSize, minSizeNode, minSizeRack, splits);
+    
+    int expectedSplitCount = (int)(totLength/maxSize);
+    Assert.assertEquals(expectedSplitCount, splits.size());
+    HashMultiset<String> nodeSplits = HashMultiset.create();
+    for(int i=0; i<expectedSplitCount; ++i) {
+      InputSplit inSplit = splits.get(i);
+      Assert.assertEquals(maxSize, inSplit.getLength());
+      Assert.assertEquals(1, inSplit.getLocations().length);
+      nodeSplits.add(inSplit.getLocations()[0]);
+    }
+    Assert.assertEquals(3, nodeSplits.count(locations[0]));
+    Assert.assertEquals(3, nodeSplits.count(locations[1]));
+  }
+  
   public void testSplitPlacementForCompressedFiles() throws Exception {
     MiniDFSCluster dfs = null;
     FileSystem fileSys = null;
@@ -889,24 +942,24 @@ public class TestCombineFileInputFormat extends TestCase {
       assertEquals(f3.getLen(), fileSplit.getLength(0));
       assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
       fileSplit = (CombineFileSplit) splits.get(1);
-      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
+      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
       assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f4.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r3
+      assertEquals(f2.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r3
       fileSplit = (CombineFileSplit) splits.get(2);
       assertEquals(1, fileSplit.getNumPaths());
       assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file2.getName(), fileSplit.getPath(0).getName());
+      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
       assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f2.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts2[0], fileSplit.getLocations()[0]); // should be on r2
+      assertEquals(f1.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r2
       fileSplit = (CombineFileSplit) splits.get(3);
       assertEquals(1, fileSplit.getNumPaths());
       assertEquals(1, fileSplit.getLocations().length);
-      assertEquals(file1.getName(), fileSplit.getPath(0).getName());
+      assertEquals(file4.getName(), fileSplit.getPath(0).getName());
       assertEquals(0, fileSplit.getOffset(0));
-      assertEquals(f1.getLen(), fileSplit.getLength(0));
-      assertEquals(hosts1[0], fileSplit.getLocations()[0]); // should be on r1
+      assertEquals(f4.getLen(), fileSplit.getLength(0));
+      assertEquals(hosts3[0], fileSplit.getLocations()[0]); // should be on r1
 
       // maximum split size is twice file1's length
       inFormat = new DummyInputFormat();

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

@@ -130,7 +130,7 @@
       <scope>test</scope>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
     <dependency>

+ 2 - 2
hadoop-mapreduce-project/pom.xml

@@ -61,7 +61,7 @@
           <artifactId>ant</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.jboss.netty</groupId>
+          <groupId>io.netty</groupId>
           <artifactId>netty</artifactId>
         </exclusion>
         <exclusion>
@@ -151,7 +151,7 @@
       <artifactId>junit</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
     <dependency>

+ 2 - 2
hadoop-project/pom.xml

@@ -391,9 +391,9 @@
       </dependency>
 
       <dependency>
-        <groupId>org.jboss.netty</groupId>
+        <groupId>io.netty</groupId>
         <artifactId>netty</artifactId>
-        <version>3.2.4.Final</version>
+        <version>3.5.11.Final</version>
       </dependency>
 
       <dependency>

+ 0 - 71
hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestUniformSizeInputFormat.java

@@ -33,8 +33,6 @@ import org.apache.hadoop.tools.CopyListing;
 import org.apache.hadoop.tools.DistCpOptions;
 import org.apache.hadoop.tools.StubContext;
 import org.apache.hadoop.security.Credentials;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -48,9 +46,6 @@ import java.util.Random;
 
 
 public class TestUniformSizeInputFormat {
-  private static final Log LOG
-                = LogFactory.getLog(TestUniformSizeInputFormat.class);
-
   private static MiniDFSCluster cluster;
   private static final int N_FILES = 20;
   private static final int SIZEOF_EACH_FILE=1024;
@@ -118,12 +113,9 @@ public class TestUniformSizeInputFormat {
     List<InputSplit> splits
             = uniformSizeInputFormat.getSplits(jobContext);
 
-    List<InputSplit> legacySplits = legacyGetSplits(listFile, nMaps);
-
     int sizePerMap = totalFileSize/nMaps;
 
     checkSplits(listFile, splits);
-    checkAgainstLegacy(splits, legacySplits);
 
     int doubleCheckedTotalSize = 0;
     int previousSplitSize = -1;
@@ -155,57 +147,6 @@ public class TestUniformSizeInputFormat {
     Assert.assertEquals(totalFileSize, doubleCheckedTotalSize);
   }
 
-  // From
-  // http://svn.apache.org/repos/asf/hadoop/mapreduce/trunk/src/tools/org/apache/hadoop/tools/DistCp.java
-  private List<InputSplit> legacyGetSplits(Path listFile, int numSplits)
-      throws IOException {
-
-    FileSystem fs = cluster.getFileSystem();
-    FileStatus srcst = fs.getFileStatus(listFile);
-    Configuration conf = fs.getConf();
-
-    ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numSplits);
-    FileStatus value = new FileStatus();
-    Text key = new Text();
-    final long targetsize = totalFileSize / numSplits;
-    long pos = 0L;
-    long last = 0L;
-    long acc = 0L;
-    long cbrem = srcst.getLen();
-    SequenceFile.Reader sl = null;
-
-    LOG.info("Average bytes per map: " + targetsize +
-        ", Number of maps: " + numSplits + ", total size: " + totalFileSize);
-
-    try {
-      sl = new SequenceFile.Reader(conf, SequenceFile.Reader.file(listFile));
-      for (; sl.next(key, value); last = sl.getPosition()) {
-        // if adding this split would put this split past the target size,
-        // cut the last split and put this next file in the next split.
-        if (acc + value.getLen() > targetsize && acc != 0) {
-          long splitsize = last - pos;
-          FileSplit fileSplit = new FileSplit(listFile, pos, splitsize, null);
-          LOG.info ("Creating split : " + fileSplit + ", bytes in split: " + splitsize);
-          splits.add(fileSplit);
-          cbrem -= splitsize;
-          pos = last;
-          acc = 0L;
-        }
-        acc += value.getLen();
-      }
-    }
-    finally {
-      IOUtils.closeStream(sl);
-    }
-    if (cbrem != 0) {
-      FileSplit fileSplit = new FileSplit(listFile, pos, cbrem, null);
-      LOG.info ("Creating split : " + fileSplit + ", bytes in split: " + cbrem);
-      splits.add(fileSplit);
-    }
-
-    return splits;
-  }
-
   private void checkSplits(Path listFile, List<InputSplit> splits) throws IOException {
     long lastEnd = 0;
 
@@ -233,18 +174,6 @@ public class TestUniformSizeInputFormat {
     }
   }
 
-  private void checkAgainstLegacy(List<InputSplit> splits,
-                                  List<InputSplit> legacySplits)
-      throws IOException, InterruptedException {
-
-    Assert.assertEquals(legacySplits.size(), splits.size());
-    for (int index = 0; index < splits.size(); index++) {
-      FileSplit fileSplit = (FileSplit) splits.get(index);
-      FileSplit legacyFileSplit = (FileSplit) legacySplits.get(index);
-      Assert.assertEquals(fileSplit.getStart(), legacyFileSplit.getStart());
-    }
-  }
-
   @Test
   public void testGetSplits() throws Exception {
     testGetSplits(9);

+ 4 - 4
hadoop-tools/hadoop-pipes/src/CMakeLists.txt

@@ -21,10 +21,10 @@ find_package(OpenSSL REQUIRED)
 
 set(CMAKE_BUILD_TYPE, Release)
 
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -g -Wall -O2")
-set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -g -Wall -O2")
-set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
-set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -D_REENTRANT -D_FILE_OFFSET_BITS=64")
+set(PIPES_FLAGS "-g -Wall -O2 -D_REENTRANT -D_GNU_SOURCE")
+set(PIPES_FLAGS "${PIPES_FLAGS} -D_LARGEFILE_SOURCE -D_FILE_OFFSET_BITS=64")
+set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${PIPES_FLAGS}")
+set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${PIPES_FLAGS}")
 
 include(../../../hadoop-common-project/hadoop-common/src/JNIFlags.cmake NO_POLICY_SCOPE)
 

+ 11 - 3
hadoop-tools/hadoop-rumen/src/main/java/org/apache/hadoop/tools/rumen/JobBuilder.java

@@ -83,6 +83,9 @@ public class JobBuilder {
   private Map<ParsedHost, ParsedHost> allHosts =
       new HashMap<ParsedHost, ParsedHost>();
 
+  private org.apache.hadoop.mapreduce.jobhistory.JhCounters EMPTY_COUNTERS =
+      new org.apache.hadoop.mapreduce.jobhistory.JhCounters();
+
   /**
    * The number of splits a task can have, before we ignore them all.
    */
@@ -459,7 +462,10 @@ public class JobBuilder {
     TaskFailed t = (TaskFailed)(event.getDatum());
     task.putDiagnosticInfo(t.error.toString());
     task.putFailedDueToAttemptId(t.failedDueToAttempt.toString());
-    // No counters in TaskFailedEvent
+    org.apache.hadoop.mapreduce.jobhistory.JhCounters counters =
+        ((TaskFailed) event.getDatum()).counters;
+    task.incorporateCounters(
+        counters == null ? EMPTY_COUNTERS : counters);
   }
 
   private void processTaskAttemptUnsuccessfulCompletionEvent(
@@ -481,7 +487,10 @@ public class JobBuilder {
     }
 
     attempt.setFinishTime(event.getFinishTime());
-
+    org.apache.hadoop.mapreduce.jobhistory.JhCounters counters =
+        ((TaskAttemptUnsuccessfulCompletion) event.getDatum()).counters;
+    attempt.incorporateCounters(
+        counters == null ? EMPTY_COUNTERS : counters);
     attempt.arraySetClockSplits(event.getClockSplits());
     attempt.arraySetCpuUsages(event.getCpuUsages());
     attempt.arraySetVMemKbytes(event.getVMemKbytes());
@@ -489,7 +498,6 @@ public class JobBuilder {
     TaskAttemptUnsuccessfulCompletion t =
         (TaskAttemptUnsuccessfulCompletion) (event.getDatum());
     attempt.putDiagnosticInfo(t.error.toString());
-    // No counters in TaskAttemptUnsuccessfulCompletionEvent
   }
 
   private void processTaskAttemptStartedEvent(TaskAttemptStartedEvent event) {

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

@@ -22,6 +22,9 @@ Release 2.0.4-beta - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-365. Change NM heartbeat handling to not generate a scheduler event
+    on each heartbeat. (Xuan Gong via sseth)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -38,6 +41,15 @@ Release 2.0.4-beta - UNRELEASED
     YARN-391. Formatting fixes for LCEResourceHandler classes.
     (Steve Loughran via sseth)
 
+    YARN-390. ApplicationCLI and NodeCLI hard-coded platform-specific line
+    separator causes test failures on Windows. (Chris Nauroth via suresh)
+
+    YARN-406. Fix TestRackResolver to function in networks where "host1"
+    resolves to a valid host. (Hitesh Shah via sseth)
+
+    YARN-376. Fixes a bug which would prevent the NM knowing about completed
+    containers and applications. (Jason Lowe via sseth)
+
 Release 2.0.3-alpha - 2013-02-06 
 
   INCOMPATIBLE CHANGES
@@ -319,6 +331,9 @@ Release 0.23.7 - UNRELEASED
     YARN-236. RM should point tracking URL to RM web page when app fails to
     start (Jason Lowe via jeagles)
 
+    YARN-269. Resource Manager not logging the health_check_script result when
+    taking it out (Jason Lowe via kihwal)
+
   OPTIMIZATIONS
 
     YARN-357. App submission should not be synchronized (daryn)
@@ -337,6 +352,9 @@ Release 0.23.7 - UNRELEASED
     YARN-400. RM can return null application resource usage report leading to 
     NPE in client (Jason Lowe via tgraves)
 
+    YARN-426. Failure to download a public resource prevents further downloads
+    (Jason Lowe via bobby)
+
 Release 0.23.6 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 19 - 7
hadoop-yarn-project/hadoop-yarn/bin/yarn

@@ -53,13 +53,7 @@ DEFAULT_LIBEXEC_DIR="$bin"/../libexec
 HADOOP_LIBEXEC_DIR=${HADOOP_LIBEXEC_DIR:-$DEFAULT_LIBEXEC_DIR}
 . $HADOOP_LIBEXEC_DIR/yarn-config.sh
 
-cygwin=false
-case "`uname`" in
-CYGWIN*) cygwin=true;;
-esac
-
-# if no args specified, show usage
-if [ $# = 0 ]; then
+function print_usage(){
   echo "Usage: yarn [--config confdir] COMMAND"
   echo "where COMMAND is one of:"
   echo "  resourcemanager      run the ResourceManager" 
@@ -76,6 +70,16 @@ if [ $# = 0 ]; then
   echo " or"
   echo "  CLASSNAME            run the class named CLASSNAME"
   echo "Most commands print help when invoked w/o parameters."
+}
+
+cygwin=false
+case "`uname`" in
+CYGWIN*) cygwin=true;;
+esac
+
+# if no args specified, show usage
+if [ $# = 0 ]; then
+  print_usage
   exit 1
 fi
 
@@ -83,6 +87,14 @@ fi
 COMMAND=$1
 shift
 
+case $COMMAND in
+  # usage flags
+  --help|-help|-h)
+    print_usage
+    exit
+    ;;
+esac
+
 if [ -f "${YARN_CONF_DIR}/yarn-env.sh" ]; then
   . "${YARN_CONF_DIR}/yarn-env.sh"
 fi

+ 34 - 27
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/ApplicationCLI.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.client.cli;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.List;
 
@@ -31,7 +33,9 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class ApplicationCLI extends YarnCLI {
-  private static final String APPLICATIONS_PATTERN = "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%35s\n";
+  private static final String APPLICATIONS_PATTERN =
+    "%30s\t%20s\t%10s\t%10s\t%18s\t%18s\t%35s" +
+    System.getProperty("line.separator");
 
   public static void main(String[] args) throws Exception {
     ApplicationCLI cli = new ApplicationCLI();
@@ -123,37 +127,40 @@ public class ApplicationCLI extends YarnCLI {
    * @throws YarnRemoteException
    */
   private void printApplicationReport(String applicationId)
-      throws YarnRemoteException {
+      throws YarnRemoteException, IOException {
     ApplicationReport appReport = client.getApplicationReport(ConverterUtils
         .toApplicationId(applicationId));
-    StringBuffer appReportStr = new StringBuffer();
+    // Use PrintWriter.println, which uses correct platform line ending.
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter appReportStr = new PrintWriter(baos);
     if (appReport != null) {
-      appReportStr.append("Application Report : ");
-      appReportStr.append("\n\tApplication-Id : ");
-      appReportStr.append(appReport.getApplicationId());
-      appReportStr.append("\n\tApplication-Name : ");
-      appReportStr.append(appReport.getName());
-      appReportStr.append("\n\tUser : ");
-      appReportStr.append(appReport.getUser());
-      appReportStr.append("\n\tQueue : ");
-      appReportStr.append(appReport.getQueue());
-      appReportStr.append("\n\tStart-Time : ");
-      appReportStr.append(appReport.getStartTime());
-      appReportStr.append("\n\tFinish-Time : ");
-      appReportStr.append(appReport.getFinishTime());
-      appReportStr.append("\n\tState : ");
-      appReportStr.append(appReport.getYarnApplicationState());
-      appReportStr.append("\n\tFinal-State : ");
-      appReportStr.append(appReport.getFinalApplicationStatus());
-      appReportStr.append("\n\tTracking-URL : ");
-      appReportStr.append(appReport.getOriginalTrackingUrl());
-      appReportStr.append("\n\tDiagnostics : ");
-      appReportStr.append(appReport.getDiagnostics());
+      appReportStr.println("Application Report : ");
+      appReportStr.print("\tApplication-Id : ");
+      appReportStr.println(appReport.getApplicationId());
+      appReportStr.print("\tApplication-Name : ");
+      appReportStr.println(appReport.getName());
+      appReportStr.print("\tUser : ");
+      appReportStr.println(appReport.getUser());
+      appReportStr.print("\tQueue : ");
+      appReportStr.println(appReport.getQueue());
+      appReportStr.print("\tStart-Time : ");
+      appReportStr.println(appReport.getStartTime());
+      appReportStr.print("\tFinish-Time : ");
+      appReportStr.println(appReport.getFinishTime());
+      appReportStr.print("\tState : ");
+      appReportStr.println(appReport.getYarnApplicationState());
+      appReportStr.print("\tFinal-State : ");
+      appReportStr.println(appReport.getFinalApplicationStatus());
+      appReportStr.print("\tTracking-URL : ");
+      appReportStr.println(appReport.getOriginalTrackingUrl());
+      appReportStr.print("\tDiagnostics : ");
+      appReportStr.print(appReport.getDiagnostics());
     } else {
-      appReportStr.append("Application with id '" + applicationId
+      appReportStr.print("Application with id '" + applicationId
           + "' doesn't exist in RM.");
     }
-    sysout.println(appReportStr.toString());
+    appReportStr.close();
+    sysout.println(baos.toString("UTF-8"));
   }
 
-}
+}

+ 35 - 28
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/NodeCLI.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.yarn.client.cli;
 
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.util.List;
 
@@ -31,7 +33,9 @@ import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 public class NodeCLI extends YarnCLI {
-  private static final String NODES_PATTERN = "%16s\t%10s\t%17s\t%26s\t%18s\n";
+  private static final String NODES_PATTERN = "%16s\t%10s\t%17s\t%26s\t%18s" +
+    System.getProperty("line.separator");
+
   public static void main(String[] args) throws Exception {
     NodeCLI cli = new NodeCLI();
     cli.setSysOutPrintStream(System.out);
@@ -100,48 +104,51 @@ public class NodeCLI extends YarnCLI {
    * @param nodeIdStr
    * @throws YarnRemoteException
    */
-  private void printNodeStatus(String nodeIdStr) throws YarnRemoteException {
+  private void printNodeStatus(String nodeIdStr) throws YarnRemoteException,
+      IOException {
     NodeId nodeId = ConverterUtils.toNodeId(nodeIdStr);
     List<NodeReport> nodesReport = client.getNodeReports();
-    StringBuffer nodeReportStr = new StringBuffer();
+    // Use PrintWriter.println, which uses correct platform line ending.
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter nodeReportStr = new PrintWriter(baos);
     NodeReport nodeReport = null;
     for (NodeReport report : nodesReport) {
       if (!report.getNodeId().equals(nodeId)) {
         continue;
       }
       nodeReport = report;
-      nodeReportStr.append("Node Report : ");
-      nodeReportStr.append("\n\tNode-Id : ");
-      nodeReportStr.append(nodeReport.getNodeId());
-      nodeReportStr.append("\n\tRack : ");
-      nodeReportStr.append(nodeReport.getRackName());
-      nodeReportStr.append("\n\tNode-State : ");
-      nodeReportStr.append(nodeReport.getNodeState());
-      nodeReportStr.append("\n\tNode-Http-Address : ");
-      nodeReportStr.append(nodeReport.getHttpAddress());
-      nodeReportStr.append("\n\tHealth-Status(isNodeHealthy) : ");
-      nodeReportStr.append(nodeReport.getNodeHealthStatus()
+      nodeReportStr.println("Node Report : ");
+      nodeReportStr.print("\tNode-Id : ");
+      nodeReportStr.println(nodeReport.getNodeId());
+      nodeReportStr.print("\tRack : ");
+      nodeReportStr.println(nodeReport.getRackName());
+      nodeReportStr.print("\tNode-State : ");
+      nodeReportStr.println(nodeReport.getNodeState());
+      nodeReportStr.print("\tNode-Http-Address : ");
+      nodeReportStr.println(nodeReport.getHttpAddress());
+      nodeReportStr.print("\tHealth-Status(isNodeHealthy) : ");
+      nodeReportStr.println(nodeReport.getNodeHealthStatus()
           .getIsNodeHealthy());
-      nodeReportStr.append("\n\tLast-Last-Health-Update : ");
-      nodeReportStr.append(nodeReport.getNodeHealthStatus()
+      nodeReportStr.print("\tLast-Last-Health-Update : ");
+      nodeReportStr.println(nodeReport.getNodeHealthStatus()
           .getLastHealthReportTime());
-      nodeReportStr.append("\n\tHealth-Report : ");
+      nodeReportStr.print("\tHealth-Report : ");
       nodeReportStr
-          .append(nodeReport.getNodeHealthStatus().getHealthReport());
-      nodeReportStr.append("\n\tContainers : ");
-      nodeReportStr.append(nodeReport.getNumContainers());
-      nodeReportStr.append("\n\tMemory-Used : ");
-      nodeReportStr.append((nodeReport.getUsed() == null) ? "0M"
+          .println(nodeReport.getNodeHealthStatus().getHealthReport());
+      nodeReportStr.print("\tContainers : ");
+      nodeReportStr.println(nodeReport.getNumContainers());
+      nodeReportStr.print("\tMemory-Used : ");
+      nodeReportStr.println((nodeReport.getUsed() == null) ? "0M"
           : (nodeReport.getUsed().getMemory() + "M"));
-      nodeReportStr.append("\n\tMemory-Capacity : ");
-      nodeReportStr.append(nodeReport.getCapability().getMemory());
+      nodeReportStr.print("\tMemory-Capacity : ");
+      nodeReportStr.println(nodeReport.getCapability().getMemory());
     }
 
     if (nodeReport == null) {
-      nodeReportStr.append("Could not find the node report for node id : "
+      nodeReportStr.print("Could not find the node report for node id : "
           + nodeIdStr);
     }
-
-    sysout.println(nodeReportStr.toString());
+    nodeReportStr.close();
+    sysout.println(baos.toString("UTF-8"));
   }
-}
+}

+ 58 - 34
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -29,6 +29,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
+import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -79,12 +80,21 @@ public class TestYarnCLI {
     int result = cli.run(new String[] { "-status", applicationId.toString() });
     assertEquals(0, result);
     verify(client).getApplicationReport(applicationId);
-    String appReportStr = "Application Report : \n\t"
-        + "Application-Id : application_1234_0005\n\t"
-        + "Application-Name : appname\n\tUser : user\n\t"
-        + "Queue : queue\n\tStart-Time : 0\n\tFinish-Time : 0\n\t"
-        + "State : FINISHED\n\tFinal-State : SUCCEEDED\n\t"
-        + "Tracking-URL : N/A\n\tDiagnostics : diagnostics\n";
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Application Report : ");
+    pw.println("\tApplication-Id : application_1234_0005");
+    pw.println("\tApplication-Name : appname");
+    pw.println("\tUser : user");
+    pw.println("\tQueue : queue");
+    pw.println("\tStart-Time : 0");
+    pw.println("\tFinish-Time : 0");
+    pw.println("\tState : FINISHED");
+    pw.println("\tFinal-State : SUCCEEDED");
+    pw.println("\tTracking-URL : N/A");
+    pw.println("\tDiagnostics : diagnostics");
+    pw.close();
+    String appReportStr = baos.toString("UTF-8");
     Assert.assertEquals(appReportStr, sysOutStream.toString());
     verify(sysOut, times(1)).println(isA(String.class));
   }
@@ -105,16 +115,18 @@ public class TestYarnCLI {
     assertEquals(0, result);
     verify(client).getApplicationList();
 
-    StringBuffer appsReportStrBuf = new StringBuffer();
-    appsReportStrBuf.append("Total Applications:1\n");
-    appsReportStrBuf
-        .append("                Application-Id\t    Application-Name"
-            + "\t      User\t     Queue\t             State\t       "
-            + "Final-State\t                       Tracking-URL\n");
-    appsReportStrBuf.append("         application_1234_0005\t             "
-        + "appname\t      user\t     queue\t          FINISHED\t         "
-        + "SUCCEEDED\t                                N/A\n");
-    Assert.assertEquals(appsReportStrBuf.toString(), sysOutStream.toString());
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Total Applications:1");
+    pw.print("                Application-Id\t    Application-Name");
+    pw.print("\t      User\t     Queue\t             State\t       ");
+    pw.println("Final-State\t                       Tracking-URL");
+    pw.print("         application_1234_0005\t             ");
+    pw.print("appname\t      user\t     queue\t          FINISHED\t         ");
+    pw.println("SUCCEEDED\t                                N/A");
+    pw.close();
+    String appsReportStr = baos.toString("UTF-8");
+    Assert.assertEquals(appsReportStr, sysOutStream.toString());
     verify(sysOut, times(1)).write(any(byte[].class), anyInt(), anyInt());
   }
 
@@ -137,18 +149,20 @@ public class TestYarnCLI {
     int result = cli.run(new String[] { "-list" });
     assertEquals(0, result);
     verify(client).getNodeReports();
-    StringBuffer nodesReportStr = new StringBuffer();
-    nodesReportStr.append("Total Nodes:3");
-    nodesReportStr
-        .append("\n         Node-Id\tNode-State\tNode-Http-Address\t"
-            + "Health-Status(isNodeHealthy)\tRunning-Containers");
-    nodesReportStr.append("\n         host0:0\t   RUNNING\t       host1:8888"
-        + "\t                     false\t                 0");
-    nodesReportStr.append("\n         host1:0\t   RUNNING\t       host1:8888"
-        + "\t                     false\t                 0");
-    nodesReportStr.append("\n         host2:0\t   RUNNING\t       host1:8888"
-        + "\t                     false\t                 0\n");
-    Assert.assertEquals(nodesReportStr.toString(), sysOutStream.toString());
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Total Nodes:3");
+    pw.print("         Node-Id\tNode-State\tNode-Http-Address\t");
+    pw.println("Health-Status(isNodeHealthy)\tRunning-Containers");
+    pw.print("         host0:0\t   RUNNING\t       host1:8888");
+    pw.println("\t                     false\t                 0");
+    pw.print("         host1:0\t   RUNNING\t       host1:8888");
+    pw.println("\t                     false\t                 0");
+    pw.print("         host2:0\t   RUNNING\t       host1:8888");
+    pw.println("\t                     false\t                 0");
+    pw.close();
+    String nodesReportStr = baos.toString("UTF-8");
+    Assert.assertEquals(nodesReportStr, sysOutStream.toString());
     verify(sysOut, times(1)).write(any(byte[].class), anyInt(), anyInt());
   }
 
@@ -163,11 +177,21 @@ public class TestYarnCLI {
     int result = cli.run(new String[] { "-status", nodeId.toString() });
     assertEquals(0, result);
     verify(client).getNodeReports();
-    String nodeStatusStr = "Node Report : \n\tNode-Id : host0:0\n\t"
-        + "Rack : rack1\n\tNode-State : RUNNING\n\t"
-        + "Node-Http-Address : host1:8888\n\tHealth-Status(isNodeHealthy) "
-        + ": false\n\tLast-Last-Health-Update : 0\n\tHealth-Report : null"
-        + "\n\tContainers : 0\n\tMemory-Used : 0M\n\tMemory-Capacity : 0";
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    PrintWriter pw = new PrintWriter(baos);
+    pw.println("Node Report : ");
+    pw.println("\tNode-Id : host0:0");
+    pw.println("\tRack : rack1");
+    pw.println("\tNode-State : RUNNING");
+    pw.println("\tNode-Http-Address : host1:8888");
+    pw.println("\tHealth-Status(isNodeHealthy) : false");
+    pw.println("\tLast-Last-Health-Update : 0");
+    pw.println("\tHealth-Report : null");
+    pw.println("\tContainers : 0");
+    pw.println("\tMemory-Used : 0M");
+    pw.println("\tMemory-Capacity : 0");
+    pw.close();
+    String nodeStatusStr = baos.toString("UTF-8");
     verify(sysOut, times(1)).println(isA(String.class));
     verify(sysOut).println(nodeStatusStr);
   }
@@ -225,4 +249,4 @@ public class TestYarnCLI {
     return cli;
   }
 
-}
+}

+ 17 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestRackResolver.java

@@ -18,9 +18,13 @@
 
 package org.apache.hadoop.yarn.util;
 
+import java.net.InetAddress;
+import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.net.DNSToSwitchMapping;
@@ -30,9 +34,12 @@ import org.junit.Test;
 
 public class TestRackResolver {
 
+  private static Log LOG = LogFactory.getLog(TestRackResolver.class);
+
   public static final class MyResolver implements DNSToSwitchMapping {
 
     int numHost1 = 0;
+    public static String resolvedHost1 = "host1";
 
     @Override
     public List<String> resolve(List<String> hostList) {
@@ -43,7 +50,10 @@ public class TestRackResolver {
       if (hostList.isEmpty()) {
         return returnList;
       }
-      if (hostList.get(0).equals("host1")) {
+      LOG.info("Received resolve request for "
+          + hostList.get(0));
+      if (hostList.get(0).equals("host1")
+          || hostList.get(0).equals(resolvedHost1)) {
         numHost1++;
         returnList.add("/rack1");
       }
@@ -62,6 +72,12 @@ public class TestRackResolver {
       CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
       MyResolver.class, DNSToSwitchMapping.class);
     RackResolver.init(conf);
+    try {
+      InetAddress iaddr = InetAddress.getByName("host1");
+      MyResolver.resolvedHost1 = iaddr.getHostAddress();
+    } catch (UnknownHostException e) {
+      // Ignore if not found
+    }
     Node node = RackResolver.resolve("host1");
     Assert.assertEquals("/rack1", node.getNetworkLocation());
     node = RackResolver.resolve("host1");

+ 11 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -659,25 +659,23 @@ public class ResourceLocalizationService extends CompositeService
                   new ContainerResourceFailedEvent(
                     assoc.getContext().getContainerId(),
                     assoc.getResource().getRequest(), e.getCause()));
+              List<LocalizerResourceRequestEvent> reqs;
               synchronized (attempts) {
                 LocalResourceRequest req = assoc.getResource().getRequest();
-                List<LocalizerResourceRequestEvent> reqs = attempts.get(req);
+                reqs = attempts.get(req);
                 if (null == reqs) {
                   LOG.error("Missing pending list for " + req);
                   return;
                 }
-                if (reqs.isEmpty()) {
-                  attempts.remove(req);
-                }
-                /* 
-                 * Do not retry for now. Once failed is failed!
-                 *  LocalizerResourceRequestEvent request = reqs.remove(0);
-
-                pending.put(queue.submit(new FSDownload(
-                    lfs, null, conf, publicDirs,
-                    request.getResource().getRequest(), new Random())),
-                    request);
-                 */              }
+                attempts.remove(req);
+              }
+              // let the other containers know about the localization failure
+              for (LocalizerResourceRequestEvent reqEvent : reqs) {
+                dispatcher.getEventHandler().handle(
+                    new ContainerResourceFailedEvent(
+                        reqEvent.getContext().getContainerId(),
+                        reqEvent.getResource().getRequest(), e.getCause()));
+              }
             } catch (CancellationException e) {
               // ignore; shutting down
             }

+ 113 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -27,13 +27,16 @@ import static org.mockito.Matchers.argThat;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Matchers.isA;
 import static org.mockito.Matchers.isNull;
+import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.timeout;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
@@ -46,6 +49,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.concurrent.BrokenBarrierException;
+import java.util.concurrent.CyclicBarrier;
 
 import junit.framework.Assert;
 
@@ -89,6 +94,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerResourceFailedEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService.LocalizerTracker;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ApplicationLocalizationEvent;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
@@ -102,6 +108,8 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.ArgumentCaptor;
 import org.mockito.ArgumentMatcher;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
 
 public class TestResourceLocalizationService {
 
@@ -512,6 +520,111 @@ public class TestResourceLocalizationService {
     }
   }
 
+  @Test(timeout=20000)
+  @SuppressWarnings("unchecked") // mocked generics
+  public void testFailedPublicResource() throws Exception {
+    Configuration conf = new YarnConfiguration();
+    AbstractFileSystem spylfs =
+      spy(FileContext.getLocalFSFileContext().getDefaultFileSystem());
+    final FileContext lfs = FileContext.getFileContext(spylfs, conf);
+    doNothing().when(spylfs).mkdir(
+        isA(Path.class), isA(FsPermission.class), anyBoolean());
+    List<Path> localDirs = new ArrayList<Path>();
+    String[] sDirs = new String[4];
+    for (int i = 0; i < 4; ++i) {
+      localDirs.add(lfs.makeQualified(new Path(basedir, i + "")));
+      sDirs[i] = localDirs.get(i).toString();
+    }
+    conf.setStrings(YarnConfiguration.NM_LOCAL_DIRS, sDirs);
+    String logDir = lfs.makeQualified(new Path(basedir, "logdir " )).toString();
+    conf.set(YarnConfiguration.NM_LOG_DIRS, logDir);
+
+    DrainDispatcher dispatcher = new DrainDispatcher();
+    EventHandler<ApplicationEvent> applicationBus = mock(EventHandler.class);
+    dispatcher.register(ApplicationEventType.class, applicationBus);
+    EventHandler<ContainerEvent> containerBus = mock(EventHandler.class);
+    dispatcher.register(ContainerEventType.class, containerBus);
+
+    ContainerExecutor exec = mock(ContainerExecutor.class);
+    DeletionService delService = mock(DeletionService.class);
+    LocalDirsHandlerService dirsHandler = new LocalDirsHandlerService();
+    dirsHandler.init(conf);
+
+    dispatcher.init(conf);
+    dispatcher.start();
+
+    try {
+      ResourceLocalizationService rawService =
+          new ResourceLocalizationService(dispatcher, exec, delService,
+                                        dirsHandler);
+      ResourceLocalizationService spyService = spy(rawService);
+      doReturn(mockServer).when(spyService).createServer();
+      doReturn(lfs).when(spyService).getLocalFileContext(
+          isA(Configuration.class));
+
+      spyService.init(conf);
+      spyService.start();
+
+      final String user = "user0";
+      // init application
+      final Application app = mock(Application.class);
+      final ApplicationId appId =
+          BuilderUtils.newApplicationId(314159265358979L, 3);
+      when(app.getUser()).thenReturn(user);
+      when(app.getAppId()).thenReturn(appId);
+      spyService.handle(new ApplicationLocalizationEvent(
+          LocalizationEventType.INIT_APPLICATION_RESOURCES, app));
+      dispatcher.await();
+
+      // init container.
+      final Container c = getMockContainer(appId, 42);
+
+      // init resources
+      Random r = new Random();
+      long seed = r.nextLong();
+      System.out.println("SEED: " + seed);
+      r.setSeed(seed);
+
+      // cause chmod to fail after a delay
+      final CyclicBarrier barrier = new CyclicBarrier(2);
+      doAnswer(new Answer<Void>() {
+          public Void answer(InvocationOnMock invocation) throws IOException {
+            try {
+              barrier.await();
+            } catch (InterruptedException e) {
+            } catch (BrokenBarrierException e) {
+            }
+            throw new IOException("forced failure");
+          }
+        }).when(spylfs)
+            .setPermission(isA(Path.class), isA(FsPermission.class));
+
+      // Queue up two localization requests for the same public resource
+      final LocalResource pubResource = getPublicMockedResource(r);
+      final LocalResourceRequest pubReq = new LocalResourceRequest(pubResource);
+
+      Map<LocalResourceVisibility, Collection<LocalResourceRequest>> req =
+          new HashMap<LocalResourceVisibility,
+                      Collection<LocalResourceRequest>>();
+      req.put(LocalResourceVisibility.PUBLIC,
+          Collections.singletonList(pubReq));
+
+      Set<LocalResourceRequest> pubRsrcs = new HashSet<LocalResourceRequest>();
+      pubRsrcs.add(pubReq);
+
+      spyService.handle(new ContainerLocalizationRequestEvent(c, req));
+      spyService.handle(new ContainerLocalizationRequestEvent(c, req));
+      dispatcher.await();
+
+      // allow the chmod to fail now that both requests have been queued
+      barrier.await();
+      verify(containerBus, timeout(5000).times(2))
+          .handle(isA(ContainerResourceFailedEvent.class));
+    } finally {
+      dispatcher.stop();
+    }
+  }
+
   private static URL getPath(String path) {
     URL url = BuilderUtils.newURL("file", null, 0, path);
     return url;

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

@@ -436,7 +436,6 @@ public class ClientRMService extends AbstractService implements
       response.setQueueInfo(queueInfo);
     } catch (IOException ioe) {
       LOG.info("Failed to getQueueInfo for " + request.getQueueName(), ioe);
-      throw RPCUtil.getRemoteException(ioe);
     }
     
     return response;

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java

@@ -262,8 +262,7 @@ public class ResourceTrackerService extends AbstractService implements
     HeartbeatResponse latestResponse = recordFactory
         .newRecordInstance(HeartbeatResponse.class);
     latestResponse.setResponseId(lastHeartbeatResponse.getResponseId() + 1);
-    latestResponse.addAllContainersToCleanup(rmNode.getContainersToCleanUp());
-    latestResponse.addAllApplicationsToCleanup(rmNode.getAppsToCleanup());
+    rmNode.updateHeartbeatResponseForCleanup(latestResponse);
     latestResponse.setNodeAction(NodeAction.NORMAL);
 
     // Check if node's masterKey needs to be updated and if the currentKey has

+ 16 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNode.java

@@ -105,5 +105,21 @@ public interface RMNode {
 
   public List<ApplicationId> getAppsToCleanup();
 
+  /**
+   * Update a {@link HeartbeatResponse} with the list of containers and
+   * applications to clean up for this node.
+   * @param response the {@link HeartbeatResponse} to update
+   */
+  public void updateHeartbeatResponseForCleanup(HeartbeatResponse response);
+
   public HeartbeatResponse getLastHeartBeatResponse();
+  
+  /**
+   * Get and clear the list of containerUpdates accumulated across NM
+   * heartbeats.
+   * 
+   * @return containerUpdates accumulated across NM heartbeats.
+   */
+  public List<UpdatedContainerInfo> pullContainerUpdates();
+  
 }

+ 58 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeImpl.java

@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -60,6 +61,8 @@ import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.BuilderUtils.ContainerIdComparator;
 
+import com.google.common.annotations.VisibleForTesting;
+
 /**
  * This class is used to keep track of all the applications/containers
  * running on a node.
@@ -78,6 +81,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
   private final ReadLock readLock;
   private final WriteLock writeLock;
 
+  private final ConcurrentLinkedQueue<UpdatedContainerInfo> nodeUpdateQueue;
+  private volatile boolean nextHeartBeat = true;
+
   private final NodeId nodeId;
   private final RMContext context;
   private final String hostName;
@@ -186,6 +192,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
 
     this.stateMachine = stateMachineFactory.make(this);
     
+    this.nodeUpdateQueue = new ConcurrentLinkedQueue<UpdatedContainerInfo>();  
   }
 
   @Override
@@ -296,6 +303,21 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     }
   };
 
+  @Override
+  public void updateHeartbeatResponseForCleanup(HeartbeatResponse response) {
+    this.writeLock.lock();
+
+    try {
+      response.addAllContainersToCleanup(
+          new ArrayList<ContainerId>(this.containersToClean));
+      response.addAllApplicationsToCleanup(this.finishedApplications);
+      this.containersToClean.clear();
+      this.finishedApplications.clear();
+    } finally {
+      this.writeLock.unlock();
+    }
+  };
+
   @Override
   public HeartbeatResponse getLastHeartBeatResponse() {
 
@@ -400,6 +422,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
       // Kill containers since node is rejoining.
+      rmNode.nodeUpdateQueue.clear();
       rmNode.context.getDispatcher().getEventHandler().handle(
           new NodeRemovedSchedulerEvent(rmNode));
 
@@ -458,6 +481,7 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
     @Override
     public void transition(RMNodeImpl rmNode, RMNodeEvent event) {
       // Inform the scheduler
+      rmNode.nodeUpdateQueue.clear();
       rmNode.context.getDispatcher().getEventHandler().handle(
           new NodeRemovedSchedulerEvent(rmNode));
       rmNode.context.getDispatcher().getEventHandler().handle(
@@ -489,6 +513,9 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           statusEvent.getNodeHealthStatus();
       rmNode.setNodeHealthStatus(remoteNodeHealthStatus);
       if (!remoteNodeHealthStatus.getIsNodeHealthy()) {
+        LOG.info("Node " + rmNode.nodeId + " reported UNHEALTHY with details: "
+            + remoteNodeHealthStatus.getHealthReport());
+        rmNode.nodeUpdateQueue.clear();
         // Inform the scheduler
         rmNode.context.getDispatcher().getEventHandler().handle(
             new NodeRemovedSchedulerEvent(rmNode));
@@ -538,20 +565,20 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
           completedContainers.add(remoteContainer);
         }
       }
-
-      rmNode.context.getDispatcher().getEventHandler().handle(
-          new NodeUpdateSchedulerEvent(rmNode, newlyLaunchedContainers, 
-              completedContainers));
+      if(newlyLaunchedContainers.size() != 0 
+          || completedContainers.size() != 0) {
+        rmNode.nodeUpdateQueue.add(new UpdatedContainerInfo
+            (newlyLaunchedContainers, completedContainers));
+      }
+      if(rmNode.nextHeartBeat) {
+        rmNode.nextHeartBeat = false;
+        rmNode.context.getDispatcher().getEventHandler().handle(
+            new NodeUpdateSchedulerEvent(rmNode));
+      }
       
       rmNode.context.getDelegationTokenRenewer().updateKeepAliveApplications(
           statusEvent.getKeepAliveAppIds());
 
-      // HeartBeat processing from our end is done, as node pulls the following
-      // lists before sending status-updates. Clear data-structures
-      // TODO: These lists could go to the NM multiple times, or never.
-      rmNode.containersToClean.clear();
-      rmNode.finishedApplications.clear();
-
       return NodeState.RUNNING;
     }
   }
@@ -584,4 +611,25 @@ public class RMNodeImpl implements RMNode, EventHandler<RMNodeEvent> {
       return NodeState.UNHEALTHY;
     }
   }
+
+  @Override
+  public List<UpdatedContainerInfo> pullContainerUpdates() {
+    List<UpdatedContainerInfo> latestContainerInfoList = 
+        new ArrayList<UpdatedContainerInfo>();
+    while(nodeUpdateQueue.peek() != null){
+      latestContainerInfoList.add(nodeUpdateQueue.poll());
+    }
+    this.nextHeartBeat = true;
+    return latestContainerInfoList;
+  }
+
+  @VisibleForTesting
+  public void setNextHeartBeat(boolean nextHeartBeat) {
+    this.nextHeartBeat = nextHeartBeat;
+  }
+  
+  @VisibleForTesting
+  public int getQueueSize() {
+    return nodeUpdateQueue.size();
+  }
  }

+ 45 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/UpdatedContainerInfo.java

@@ -0,0 +1,45 @@
+/**
+ * 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.rmnode;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+
+public class UpdatedContainerInfo {
+  private List<ContainerStatus> newlyLaunchedContainers;
+  private List<ContainerStatus> completedContainers;
+  
+  public UpdatedContainerInfo() {
+  }
+
+  public UpdatedContainerInfo(List<ContainerStatus> newlyLaunchedContainers
+      , List<ContainerStatus> completedContainers) {
+    this.newlyLaunchedContainers = newlyLaunchedContainers;
+    this.completedContainers = completedContainers;
+  } 
+
+  public List<ContainerStatus> getNewlyLaunchedContainers() {
+    return this.newlyLaunchedContainers;
+  }
+
+  public List<ContainerStatus> getCompletedContainers() {
+    return this.completedContainers;
+  }
+}

+ 12 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
@@ -562,15 +563,20 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     return root.getQueueUserAclInfo(user);
   }
 
-  private synchronized void nodeUpdate(RMNode nm, 
-      List<ContainerStatus> newlyLaunchedContainers,
-      List<ContainerStatus> completedContainers) {
+  private synchronized void nodeUpdate(RMNode nm) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("nodeUpdate: " + nm + " clusterResources: " + clusterResource);
     }
-                  
-    FiCaSchedulerNode node = getNode(nm.getNodeID());
 
+    FiCaSchedulerNode node = getNode(nm.getNodeID());
+    List<UpdatedContainerInfo> containerInfoList = nm.pullContainerUpdates();
+    List<ContainerStatus> newlyLaunchedContainers = new ArrayList<ContainerStatus>();
+    List<ContainerStatus> completedContainers = new ArrayList<ContainerStatus>();
+    for(UpdatedContainerInfo containerInfo : containerInfoList) {
+      newlyLaunchedContainers.addAll(containerInfo.getNewlyLaunchedContainers());
+      completedContainers.addAll(containerInfo.getCompletedContainers());
+    }
+    
     // Processing the newly launched containers
     for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
       containerLaunchedOnNode(launchedContainer.getContainerId(), node);
@@ -666,9 +672,7 @@ implements ResourceScheduler, CapacitySchedulerContext, Configurable {
     case NODE_UPDATE:
     {
       NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
-      nodeUpdate(nodeUpdatedEvent.getRMNode(), 
-          nodeUpdatedEvent.getNewlyLaunchedContainers(),
-          nodeUpdatedEvent.getCompletedContainers());
+      nodeUpdate(nodeUpdatedEvent.getRMNode());
     }
     break;
     case APP_ADDED:

+ 1 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/event/NodeUpdateSchedulerEvent.java

@@ -18,35 +18,18 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.scheduler.event;
 
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 
 public class NodeUpdateSchedulerEvent extends SchedulerEvent {
 
   private final RMNode rmNode;
-  private final List<ContainerStatus> newlyLaunchedContainers;
-  private final List<ContainerStatus> completedContainersStatuses;
 
-  public NodeUpdateSchedulerEvent(RMNode rmNode,
-      List<ContainerStatus> newlyLaunchedContainers,
-      List<ContainerStatus> completedContainers) {
+  public NodeUpdateSchedulerEvent(RMNode rmNode) {
     super(SchedulerEventType.NODE_UPDATE);
     this.rmNode = rmNode;
-    this.newlyLaunchedContainers = newlyLaunchedContainers;
-    this.completedContainersStatuses = completedContainers;
   }
 
   public RMNode getRMNode() {
     return rmNode;
   }
-
-  public List<ContainerStatus> getNewlyLaunchedContainers() {
-    return newlyLaunchedContainers;
-  }
-
-  public List<ContainerStatus> getCompletedContainers() {
-    return completedContainersStatuses;
-  }
 }

+ 11 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java

@@ -33,7 +33,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.SystemClock;
@@ -61,6 +60,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics;
@@ -750,15 +750,20 @@ public class FairScheduler implements ResourceScheduler {
   /**
    * Process a heartbeat update from a node.
    */
-  private synchronized void nodeUpdate(RMNode nm,
-      List<ContainerStatus> newlyLaunchedContainers,
-      List<ContainerStatus> completedContainers) {
+  private synchronized void nodeUpdate(RMNode nm) {
     if (LOG.isDebugEnabled()) {
       LOG.debug("nodeUpdate: " + nm + " cluster capacity: " + clusterCapacity);
     }
     eventLog.log("HEARTBEAT", nm.getHostName());
     FSSchedulerNode node = nodes.get(nm.getNodeID());
 
+    List<UpdatedContainerInfo> containerInfoList = nm.pullContainerUpdates();
+    List<ContainerStatus> newlyLaunchedContainers = new ArrayList<ContainerStatus>();
+    List<ContainerStatus> completedContainers = new ArrayList<ContainerStatus>();
+    for(UpdatedContainerInfo containerInfo : containerInfoList) {
+      newlyLaunchedContainers.addAll(containerInfo.getNewlyLaunchedContainers());
+      completedContainers.addAll(containerInfo.getCompletedContainers());
+    } 
     // Processing the newly launched containers
     for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
       containerLaunchedOnNode(launchedContainer.getContainerId(), node);
@@ -864,9 +869,7 @@ public class FairScheduler implements ResourceScheduler {
         throw new RuntimeException("Unexpected event type: " + event);
       }
       NodeUpdateSchedulerEvent nodeUpdatedEvent = (NodeUpdateSchedulerEvent)event;
-      nodeUpdate(nodeUpdatedEvent.getRMNode(),
-          nodeUpdatedEvent.getNewlyLaunchedContainers(),
-          nodeUpdatedEvent.getCompletedContainers());
+      nodeUpdate(nodeUpdatedEvent.getRMNode());
       break;
     case APP_ADDED:
       if (!(event instanceof AppAddedSchedulerEvent)) {
@@ -966,7 +969,7 @@ public class FairScheduler implements ResourceScheduler {
   public QueueInfo getQueueInfo(String queueName, boolean includeChildQueues,
       boolean recursive) throws IOException {
     if (!queueMgr.exists(queueName)) {
-      return null;
+      throw new IOException("queue " + queueName + " does not exist");
     }
     return queueMgr.getQueue(queueName).getQueueInfo(includeChildQueues,
         recursive);

+ 10 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -67,6 +67,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
@@ -576,11 +577,16 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
     return assignedContainers;
   }
 
-  private synchronized void nodeUpdate(RMNode rmNode, 
-      List<ContainerStatus> newlyLaunchedContainers,
-      List<ContainerStatus> completedContainers) {
+  private synchronized void nodeUpdate(RMNode rmNode) {
     FiCaSchedulerNode node = getNode(rmNode.getNodeID());
     
+    List<UpdatedContainerInfo> containerInfoList = rmNode.pullContainerUpdates();
+    List<ContainerStatus> newlyLaunchedContainers = new ArrayList<ContainerStatus>();
+    List<ContainerStatus> completedContainers = new ArrayList<ContainerStatus>();
+    for(UpdatedContainerInfo containerInfo : containerInfoList) {
+      newlyLaunchedContainers.addAll(containerInfo.getNewlyLaunchedContainers());
+      completedContainers.addAll(containerInfo.getCompletedContainers());
+    }
     // Processing the newly launched containers
     for (ContainerStatus launchedContainer : newlyLaunchedContainers) {
       containerLaunchedOnNode(launchedContainer.getContainerId(), node);
@@ -628,9 +634,7 @@ public class FifoScheduler implements ResourceScheduler, Configurable {
     {
       NodeUpdateSchedulerEvent nodeUpdatedEvent = 
       (NodeUpdateSchedulerEvent)event;
-      nodeUpdate(nodeUpdatedEvent.getRMNode(), 
-          nodeUpdatedEvent.getNewlyLaunchedContainers(),
-          nodeUpdatedEvent.getCompletedContainers());
+      nodeUpdate(nodeUpdatedEvent.getRMNode());
     }
     break;
     case APP_ADDED:

+ 12 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNodes.java

@@ -18,7 +18,9 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.hadoop.net.Node;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -31,6 +33,7 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 
 import com.google.common.collect.Lists;
 
@@ -183,10 +186,19 @@ public class MockNodes {
       return null;
     }
 
+    @Override
+    public void updateHeartbeatResponseForCleanup(HeartbeatResponse response) {
+    }
+
     @Override
     public HeartbeatResponse getLastHeartBeatResponse() {
       return null;
     }
+
+    @Override
+    public List<UpdatedContainerInfo> pullContainerUpdates() {
+      return new ArrayList<UpdatedContainerInfo>();
+    }
   };
 
   private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -19,7 +19,7 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Matchers.anyString;
+import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
@@ -176,6 +176,10 @@ public class TestClientRMService {
     List<ApplicationReport> applications = queueInfo.getQueueInfo()
         .getApplications();
     Assert.assertEquals(2, applications.size());
+    request.setQueueName("nonexistentqueue");
+    request.setIncludeApplications(true);
+    // should not throw exception on nonexistent queue
+    queueInfo = rmService.getQueueInfo(request);
   }
 
   private static final UserGroupInformation owner =
@@ -334,8 +338,10 @@ public class TestClientRMService {
     when(rmContext.getDispatcher()).thenReturn(dispatcher);
     QueueInfo queInfo = recordFactory.newRecordInstance(QueueInfo.class);
     queInfo.setQueueName("testqueue");
-    when(yarnScheduler.getQueueInfo(anyString(), anyBoolean(), anyBoolean()))
+    when(yarnScheduler.getQueueInfo(eq("testqueue"), anyBoolean(), anyBoolean()))
         .thenReturn(queInfo);
+    when(yarnScheduler.getQueueInfo(eq("nonexistentqueue"), anyBoolean(), anyBoolean()))
+        .thenThrow(new IOException("queue does not exist"));
     ConcurrentHashMap<ApplicationId, RMApp> apps = getRMApps(rmContext,
         yarnScheduler);
     when(rmContext.getRMApps()).thenReturn(apps);

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

@@ -201,7 +201,7 @@ public class TestFifoScheduler {
     testMinimumAllocation(conf, allocMB / 2);
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testReconnectedNode() throws Exception {
     CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration();
     conf.setQueues("default", new String[] {"default"});
@@ -215,19 +215,19 @@ public class TestFifoScheduler {
     fs.handle(new NodeAddedSchedulerEvent(n1));
     fs.handle(new NodeAddedSchedulerEvent(n2));
     List<ContainerStatus> emptyList = new ArrayList<ContainerStatus>();
-    fs.handle(new NodeUpdateSchedulerEvent(n1, emptyList, emptyList));
+    fs.handle(new NodeUpdateSchedulerEvent(n1));
     Assert.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB());
 
     // reconnect n1 with downgraded memory
     n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 1);
     fs.handle(new NodeRemovedSchedulerEvent(n1));
     fs.handle(new NodeAddedSchedulerEvent(n1));
-    fs.handle(new NodeUpdateSchedulerEvent(n1, emptyList, emptyList));
+    fs.handle(new NodeUpdateSchedulerEvent(n1));
 
     Assert.assertEquals(4 * GB, fs.getRootQueueMetrics().getAvailableMB());
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testHeadroom() throws Exception {
     
     Configuration conf = new Configuration();
@@ -275,7 +275,7 @@ public class TestFifoScheduler {
     fs.allocate(appAttemptId2, ask2, emptyId);
     
     // Trigger container assignment
-    fs.handle(new NodeUpdateSchedulerEvent(n1, emptyStatus, emptyStatus));
+    fs.handle(new NodeUpdateSchedulerEvent(n1));
     
     // Get the allocation for the applications and verify headroom
     Allocation allocation1 = fs.allocate(appAttemptId1, emptyAsk, emptyId);

+ 150 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMNodeTransitions.java

@@ -22,6 +22,7 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.*;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -29,6 +30,7 @@ import java.util.List;
 
 import junit.framework.Assert;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
@@ -37,11 +39,13 @@ import org.apache.hadoop.yarn.api.records.NodeState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.apache.hadoop.yarn.server.api.records.HeartbeatResponse;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeCleanContainerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeStatusEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.rmnode.UpdatedContainerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@@ -49,6 +53,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
 import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.apache.hadoop.yarn.util.Records;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -63,7 +68,7 @@ public class TestRMNodeTransitions {
   private YarnScheduler scheduler;
 
   private SchedulerEventType eventType;
-  private List<ContainerStatus> completedContainers;
+  private List<ContainerStatus> completedContainers = new ArrayList<ContainerStatus>();
   
   private final class TestSchedulerEventDispatcher implements
   EventHandler<SchedulerEvent> {
@@ -89,10 +94,11 @@ public class TestRMNodeTransitions {
             final SchedulerEvent event = (SchedulerEvent)(invocation.getArguments()[0]);
             eventType = event.getType();
             if (eventType == SchedulerEventType.NODE_UPDATE) {
-              completedContainers = 
-                  ((NodeUpdateSchedulerEvent)event).getCompletedContainers();
-            } else {
-              completedContainers = null;
+              List<UpdatedContainerInfo> lastestContainersInfoList = 
+                  ((NodeUpdateSchedulerEvent)event).getRMNode().pullContainerUpdates();
+              for(UpdatedContainerInfo lastestContainersInfo : lastestContainersInfoList) {
+            	  completedContainers.addAll(lastestContainersInfo.getCompletedContainers()); 
+              }
             }
             return null;
           }
@@ -125,16 +131,16 @@ public class TestRMNodeTransitions {
     return event;
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testExpiredContainer() {
     // Start the node
     node.handle(new RMNodeEvent(null, RMNodeEventType.STARTED));
     verify(scheduler).handle(any(NodeAddedSchedulerEvent.class));
     
     // Expire a container
-		ContainerId completedContainerId = BuilderUtils.newContainerId(
-				BuilderUtils.newApplicationAttemptId(
-						BuilderUtils.newApplicationId(0, 0), 0), 0);
+    ContainerId completedContainerId = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(0, 0), 0), 0);
     node.handle(new RMNodeCleanContainerEvent(null, completedContainerId));
     Assert.assertEquals(1, node.getContainersToCleanUp().size());
     
@@ -146,9 +152,110 @@ public class TestRMNodeTransitions {
     doReturn(Collections.singletonList(containerStatus)).
         when(statusEvent).getContainers();
     node.handle(statusEvent);
-    Assert.assertEquals(0, completedContainers.size());
+    /* Expect the scheduler call handle function 2 times
+     * 1. RMNode status from new to Running, handle the add_node event
+     * 2. handle the node update event
+     */
+    verify(scheduler,times(2)).handle(any(NodeUpdateSchedulerEvent.class));     
+  }
+  
+  @Test (timeout = 5000)
+  public void testContainerUpdate() throws InterruptedException{
+    //Start the node
+    node.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    
+    NodeId nodeId = BuilderUtils.newNodeId("localhost:1", 1);
+    RMNodeImpl node2 = new RMNodeImpl(nodeId, rmContext, null, 0, 0, null, null);
+    node2.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    
+    ContainerId completedContainerIdFromNode1 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(0, 0), 0), 0);
+    ContainerId completedContainerIdFromNode2_1 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(1, 1), 1), 1);
+    ContainerId completedContainerIdFromNode2_2 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(1, 1), 1), 2);
+ 
+    RMNodeStatusEvent statusEventFromNode1 = getMockRMNodeStatusEvent();
+    RMNodeStatusEvent statusEventFromNode2_1 = getMockRMNodeStatusEvent();
+    RMNodeStatusEvent statusEventFromNode2_2 = getMockRMNodeStatusEvent();
+    
+    ContainerStatus containerStatusFromNode1 = mock(ContainerStatus.class);
+    ContainerStatus containerStatusFromNode2_1 = mock(ContainerStatus.class);
+    ContainerStatus containerStatusFromNode2_2 = mock(ContainerStatus.class);
+
+    doReturn(completedContainerIdFromNode1).when(containerStatusFromNode1)
+        .getContainerId();
+    doReturn(Collections.singletonList(containerStatusFromNode1))
+        .when(statusEventFromNode1).getContainers();
+    node.handle(statusEventFromNode1);
+    Assert.assertEquals(1, completedContainers.size());
+    Assert.assertEquals(completedContainerIdFromNode1,
+        completedContainers.get(0).getContainerId());
+
+    completedContainers.clear();
+
+    doReturn(completedContainerIdFromNode2_1).when(containerStatusFromNode2_1)
+        .getContainerId();
+    doReturn(Collections.singletonList(containerStatusFromNode2_1))
+        .when(statusEventFromNode2_1).getContainers();
+
+    doReturn(completedContainerIdFromNode2_2).when(containerStatusFromNode2_2)
+        .getContainerId();
+    doReturn(Collections.singletonList(containerStatusFromNode2_2))
+        .when(statusEventFromNode2_2).getContainers();
+
+    node2.setNextHeartBeat(false);
+    node2.handle(statusEventFromNode2_1);
+    node2.setNextHeartBeat(true);
+    node2.handle(statusEventFromNode2_2);
+
+    Assert.assertEquals(2, completedContainers.size());
+    Assert.assertEquals(completedContainerIdFromNode2_1,completedContainers.get(0)
+        .getContainerId()); 
+    Assert.assertEquals(completedContainerIdFromNode2_2,completedContainers.get(1)
+        .getContainerId());   
   }
+  
+  @Test (timeout = 5000)
+  public void testStatusChange(){
+    //Start the node
+    node.handle(new RMNodeEvent(null,RMNodeEventType.STARTED));
+    //Add info to the queue first
+    node.setNextHeartBeat(false);
+
+    ContainerId completedContainerId1 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(0, 0), 0), 0);
+    ContainerId completedContainerId2 = BuilderUtils.newContainerId(
+        BuilderUtils.newApplicationAttemptId(
+            BuilderUtils.newApplicationId(1, 1), 1), 1);
+        
+    RMNodeStatusEvent statusEvent1 = getMockRMNodeStatusEvent();
+    RMNodeStatusEvent statusEvent2 = getMockRMNodeStatusEvent();
+
+    ContainerStatus containerStatus1 = mock(ContainerStatus.class);
+    ContainerStatus containerStatus2 = mock(ContainerStatus.class);
 
+    doReturn(completedContainerId1).when(containerStatus1).getContainerId();
+    doReturn(Collections.singletonList(containerStatus1))
+        .when(statusEvent1).getContainers();
+     
+    doReturn(completedContainerId2).when(containerStatus2).getContainerId();
+    doReturn(Collections.singletonList(containerStatus2))
+        .when(statusEvent2).getContainers();
+
+    verify(scheduler,times(1)).handle(any(NodeUpdateSchedulerEvent.class)); 
+    node.handle(statusEvent1);
+    node.handle(statusEvent2);
+    verify(scheduler,times(1)).handle(any(NodeUpdateSchedulerEvent.class));
+    Assert.assertEquals(2, node.getQueueSize());
+    node.handle(new RMNodeEvent(node.getNodeID(), RMNodeEventType.EXPIRE));
+    Assert.assertEquals(0, node.getQueueSize());
+  }
+  
   @Test
   public void testRunningExpire() {
     RMNodeImpl node = getRunningNode();
@@ -195,6 +302,39 @@ public class TestRMNodeTransitions {
     Assert.assertEquals(NodeState.REBOOTED, node.getState());
   }
 
+  @Test(timeout=20000)
+  public void testUpdateHeartbeatResponseForCleanup() {
+    RMNodeImpl node = getRunningNode();
+    NodeId nodeId = node.getNodeID();
+
+    // Expire a container
+		ContainerId completedContainerId = BuilderUtils.newContainerId(
+				BuilderUtils.newApplicationAttemptId(
+						BuilderUtils.newApplicationId(0, 0), 0), 0);
+    node.handle(new RMNodeCleanContainerEvent(nodeId, completedContainerId));
+    Assert.assertEquals(1, node.getContainersToCleanUp().size());
+
+    // Finish an application
+    ApplicationId finishedAppId = BuilderUtils.newApplicationId(0, 1);
+    node.handle(new RMNodeCleanAppEvent(nodeId, finishedAppId));
+    Assert.assertEquals(1, node.getAppsToCleanup().size());
+
+    // Verify status update does not clear containers/apps to cleanup
+    // but updating heartbeat response for cleanup does
+    RMNodeStatusEvent statusEvent = getMockRMNodeStatusEvent();
+    node.handle(statusEvent);
+    Assert.assertEquals(1, node.getContainersToCleanUp().size());
+    Assert.assertEquals(1, node.getAppsToCleanup().size());
+    HeartbeatResponse hbrsp = Records.newRecord(HeartbeatResponse.class);
+    node.updateHeartbeatResponseForCleanup(hbrsp);
+    Assert.assertEquals(0, node.getContainersToCleanUp().size());
+    Assert.assertEquals(0, node.getAppsToCleanup().size());
+    Assert.assertEquals(1, hbrsp.getContainersToCleanupCount());
+    Assert.assertEquals(completedContainerId, hbrsp.getContainerToCleanup(0));
+    Assert.assertEquals(1, hbrsp.getApplicationsToCleanupCount());
+    Assert.assertEquals(finishedAppId, hbrsp.getApplicationsToCleanup(0));
+  }
+
   private RMNodeImpl getRunningNode() {
     NodeId nodeId = BuilderUtils.newNodeId("localhost", 0);
     RMNodeImpl node = new RMNodeImpl(nodeId, rmContext,null, 0, 0,

+ 29 - 48
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -30,7 +30,6 @@ import java.io.PrintWriter;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
@@ -276,7 +275,7 @@ public class TestFairScheduler {
     Assert.assertEquals(3, queueManager.getLeafQueues().size());
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testSimpleContainerAllocation() {
     // Add a node
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
@@ -292,8 +291,7 @@ public class TestFairScheduler {
 
     scheduler.update();
 
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
 
     // Asked for less than min_allocation.
@@ -301,15 +299,14 @@ public class TestFairScheduler {
         scheduler.getQueueManager().getQueue("queue1").
         getResourceUsage().getMemory());
 
-    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
-        new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
     scheduler.handle(updateEvent2);
 
     assertEquals(1024, scheduler.getQueueManager().getQueue("queue1").
       getResourceUsage().getMemory());
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testSimpleContainerReservation() throws InterruptedException {
     // Add a node
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
@@ -319,8 +316,7 @@ public class TestFairScheduler {
     // Queue 1 requests full capacity of node
     createSchedulingRequest(1024, "queue1", "user1", 1);
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
 
     // Make sure queue 1 is allocated app capacity
@@ -340,8 +336,7 @@ public class TestFairScheduler {
     // Now another node checks in with capacity
     RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
     NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
-    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
-        new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
     scheduler.handle(nodeEvent2);
     scheduler.handle(updateEvent2);
 
@@ -738,7 +733,7 @@ public class TestFairScheduler {
     assertEquals(300000, queueManager.getFairSharePreemptionTimeout());
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testIsStarvedForMinShare() throws Exception {
     Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
@@ -767,8 +762,7 @@ public class TestFairScheduler {
     // Queue A wants 3 * 1024. Node update gives this all to A
     createSchedulingRequest(3 * 1024, "queueA", "user1");
     scheduler.update();
-    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1,
-        new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(nodeEvent2);
 
     // Queue B arrives and wants 1 * 1024
@@ -797,7 +791,7 @@ public class TestFairScheduler {
     }
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testIsStarvedForFairShare() throws Exception {
     Configuration conf = createConfiguration();
     conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
@@ -826,8 +820,7 @@ public class TestFairScheduler {
     // Queue A wants 3 * 1024. Node update gives this all to A
     createSchedulingRequest(3 * 1024, "queueA", "user1");
     scheduler.update();
-    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1,
-        new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+    NodeUpdateSchedulerEvent nodeEvent2 = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(nodeEvent2);
 
     // Queue B arrives and wants 1 * 1024
@@ -857,7 +850,7 @@ public class TestFairScheduler {
     }
   }
 
-  @Test
+  @Test (timeout = 5000)
   /**
    * Make sure containers are chosen to be preempted in the correct order. Right
    * now this means decreasing order of priority.
@@ -921,16 +914,13 @@ public class TestFairScheduler {
 
     // Sufficient node check-ins to fully schedule containers
     for (int i = 0; i < 2; i++) {
-      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
       scheduler.handle(nodeUpdate1);
 
-      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
       scheduler.handle(nodeUpdate2);
 
-      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
       scheduler.handle(nodeUpdate3);
     }
 
@@ -991,7 +981,7 @@ public class TestFairScheduler {
     assertEquals(0, scheduler.applications.get(app6).getLiveContainers().size());
   }
 
-  @Test
+  @Test (timeout = 5000)
   /**
    * Tests the timing of decision to preempt tasks.
    */
@@ -1062,16 +1052,13 @@ public class TestFairScheduler {
 
     // Sufficient node check-ins to fully schedule containers
     for (int i = 0; i < 2; i++) {
-      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate1 = new NodeUpdateSchedulerEvent(node1);
       scheduler.handle(nodeUpdate1);
 
-      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate2 = new NodeUpdateSchedulerEvent(node2);
       scheduler.handle(nodeUpdate2);
 
-      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3,
-          new LinkedList<ContainerStatus>(), new LinkedList<ContainerStatus>());
+      NodeUpdateSchedulerEvent nodeUpdate3 = new NodeUpdateSchedulerEvent(node3);
       scheduler.handle(nodeUpdate3);
     }
 
@@ -1119,7 +1106,7 @@ public class TestFairScheduler {
         Resources.createResource(1536), scheduler.resToPreempt(schedD, clock.getTime())));
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testMultipleContainersWaitingForReservation() {
     // Add a node
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
@@ -1129,8 +1116,7 @@ public class TestFairScheduler {
     // Request full capacity of node
     createSchedulingRequest(1024, "queue1", "user1", 1);
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
 
     ApplicationAttemptId attId1 = createSchedulingRequest(1024, "queue2", "user2", 1);
@@ -1146,7 +1132,7 @@ public class TestFairScheduler {
         scheduler.applications.get(attId2).getCurrentReservation().getMemory());
   }
 
-  @Test
+  @Test (timeout = 5000)
   public void testUserMaxRunningApps() throws Exception {
     // Set max running apps
     Configuration conf = createConfiguration();
@@ -1175,8 +1161,7 @@ public class TestFairScheduler {
         "user1", 1);
     
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
     
     // App 1 should be running
@@ -1201,7 +1186,7 @@ public class TestFairScheduler {
     assertEquals(2, scheduler.applications.get(attId1).getLiveContainers().size());
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testReservationWhileMultiplePriorities() {
     // Add a node
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
@@ -1211,8 +1196,7 @@ public class TestFairScheduler {
     ApplicationAttemptId attId = createSchedulingRequest(1024, "queue1",
         "user1", 1, 2);
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent);
     
     FSSchedulerApp app = scheduler.applications.get(attId);
@@ -1285,7 +1269,7 @@ public class TestFairScheduler {
     assertNull("The application was allowed", app2);
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testMultipleNodesSingleRackRequest() throws Exception {
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
     RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024));
@@ -1312,22 +1296,20 @@ public class TestFairScheduler {
     
     // node 1 checks in
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent1 = new NodeUpdateSchedulerEvent(node1);
     scheduler.handle(updateEvent1);
     // should assign node local
     assertEquals(1, scheduler.applications.get(appId).getLiveContainers().size());
 
     // node 2 checks in
     scheduler.update();
-    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
     scheduler.handle(updateEvent2);
     // should assign rack local
     assertEquals(2, scheduler.applications.get(appId).getLiveContainers().size());
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testFifoWithinQueue() throws Exception {
     RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(3072));
     NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
@@ -1351,8 +1333,7 @@ public class TestFairScheduler {
     // Because tests set assignmultiple to false, each heartbeat assigns a single
     // container.
     
-    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1,
-      new ArrayList<ContainerStatus>(), new ArrayList<ContainerStatus>());
+    NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
 
     scheduler.handle(updateEvent);
     assertEquals(1, app1.getLiveContainers().size());

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

@@ -85,7 +85,7 @@
       <artifactId>guice-servlet</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
     <dependency>

+ 2 - 2
hadoop-yarn-project/pom.xml

@@ -59,7 +59,7 @@
           <artifactId>ant</artifactId>
         </exclusion>
         <exclusion>
-          <groupId>org.jboss.netty</groupId>
+          <groupId>io.netty</groupId>
           <artifactId>netty</artifactId>
         </exclusion>
         <exclusion>
@@ -149,7 +149,7 @@
       <artifactId>junit</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.jboss.netty</groupId>
+      <groupId>io.netty</groupId>
       <artifactId>netty</artifactId>
     </dependency>
     <dependency>