Browse Source

Merge trunk into HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1173012 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 years ago
parent
commit
c382e386b5
91 changed files with 2485 additions and 715 deletions
  1. 9 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 3 0
      hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh
  3. 1 0
      hadoop-common-project/hadoop-common/src/main/packages/rpm/spec/hadoop.spec
  4. 0 6
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml
  5. 11 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  6. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  7. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  8. 181 134
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  9. 4 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImagePreTransactionalStorageInspector.java
  10. 32 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  11. 7 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  12. 121 57
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  14. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  15. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHDFSConcat.java
  16. 40 0
      hadoop-mapreduce-project/CHANGES.txt
  17. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
  18. 171 81
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  19. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java
  20. 15 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  21. 96 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java
  22. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java
  23. 20 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  24. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  25. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
  26. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRApp.java
  27. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
  28. 84 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobId.java
  29. 80 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java
  30. 101 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskId.java
  31. 11 35
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobIdPBImpl.java
  32. 12 40
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskAttemptIdPBImpl.java
  33. 15 42
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskIdPBImpl.java
  34. 56 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java
  35. 139 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestIds.java
  36. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java
  37. 8 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/ReduceTask.java
  38. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java
  39. 115 26
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
  40. 0 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java
  41. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRConfig.java
  42. 6 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java
  43. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java
  44. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobMonitorAndPrint.java
  45. 2 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
  46. 86 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java
  47. 5 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
  48. 28 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  49. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ResourceMgrDelegate.java
  50. 6 25
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  51. 2 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YarnClientProtocolProvider.java
  52. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
  53. 166 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  54. 6 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
  55. 8 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  56. 31 32
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java
  57. 31 13
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java
  58. 21 31
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
  59. 50 5
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java
  60. 5 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java
  61. 7 59
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java
  62. 21 14
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java
  63. 5 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java
  64. 29 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java
  65. 23 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java
  66. 25 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java
  67. 56 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java
  68. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
  69. 80 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
  70. 15 3
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  71. 71 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
  72. 12 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  73. 3 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
  74. 4 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
  75. 5 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
  76. 5 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
  77. 30 1
      hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java
  78. 11 0
      hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java
  79. 32 0
      hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/UndeclaredPoolException.java
  80. 182 0
      hadoop-mapreduce-project/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerPoolNames.java
  81. 9 0
      hadoop-mapreduce-project/src/docs/src/documentation/content/xdocs/fair_scheduler.xml
  82. 1 1
      hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java
  83. 1 1
      hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java
  84. 1 1
      hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java
  85. 1 1
      hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java
  86. 7 0
      hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java
  87. 10 0
      hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/TaskScheduler.java
  88. 1 1
      hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestClusterStatus.java
  89. 1 1
      hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobCounters.java
  90. 5 5
      hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestMapCollection.java
  91. 3 5
      hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java

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

@@ -594,6 +594,15 @@ Release 0.23.0 - Unreleased
     HADOOP-7608. SnappyCodec check for Hadoop native lib is wrong
     (Alejandro Abdelnur via todd)
 
+    HADOOP-7637. Fix to include FairScheduler configuration file in
+    RPM. (Eric Yang via ddas)
+
+    HADOOP-7633. Adds log4j.properties to the hadoop-conf dir on
+    deploy (Eric Yang via ddas)
+
+    HADOOP-7631. Fixes a config problem to do with running streaming jobs
+    (Eric Yang via ddas)
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 3 - 0
hadoop-common-project/hadoop-common/src/main/packages/hadoop-setup-conf.sh

@@ -477,6 +477,9 @@ else
   if [ ! -e ${HADOOP_CONF_DIR}/capacity-scheduler.xml ]; then
     template_generator ${HADOOP_PREFIX}/share/hadoop/templates/conf/capacity-scheduler.xml ${HADOOP_CONF_DIR}/capacity-scheduler.xml
   fi
+  if [ ! -e ${HADOOP_CONF_DIR}/log4j.properties ]; then
+    cp ${HADOOP_PREFIX}/share/hadoop/common/templates/conf/log4j.properties ${HADOOP_CONF_DIR}/log4j.properties
+  fi
   
   chown root:${HADOOP_GROUP} ${HADOOP_CONF_DIR}/hadoop-env.sh
   chmod 755 ${HADOOP_CONF_DIR}/hadoop-env.sh

+ 1 - 0
hadoop-common-project/hadoop-common/src/main/packages/rpm/spec/hadoop.spec

@@ -166,6 +166,7 @@ bash ${RPM_INSTALL_PREFIX0}/sbin/update-hadoop-env.sh \
 %config(noreplace) %{_conf_dir}/log4j.properties
 %config(noreplace) %{_conf_dir}/masters
 %config(noreplace) %{_conf_dir}/slaves
+%config{noreplace) %{_conf_dir}/fair-scheduler.xml
 %{_conf_dir}/hadoop-env.sh.template
 %{_conf_dir}/ssl-client.xml.example
 %{_conf_dir}/ssl-server.xml.example

+ 0 - 6
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml

@@ -103,12 +103,6 @@
     <value>INFO,TLA</value>
   </property>
 
-  <property>
-    <name>stream.tmpdir</name>
-    <value>${mapred.temp.dir}</value>
-  </property>
-
-
   <property>
     <name>mapred.child.java.opts</name>
     <value>-server -Xmx640m -Djava.net.preferIPv4Stack=true</value>

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

@@ -31,6 +31,10 @@ Trunk (unreleased changes)
 
     HDFS-2337. DFSClient shouldn't keep multiple RPC proxy references (atm)
 
+    HDFS-362.  FSEditLog should not writes long and short as UTF8, and should
+    not use ArrayWritable for writing non-array items.  (Uma Maheswara Rao G
+    via szetszwo)
+
   BUG FIXES
     HDFS-2287. TestParallelRead has a small off-by-one bug. (todd)
 
@@ -49,6 +53,10 @@ Trunk (unreleased changes)
     HDFS-2333. Change DFSOutputStream back to package private, otherwise,
     there are two SC_START_IN_CTOR findbugs warnings.  (szetszwo)
 
+    HDFS-2330. In NNStorage and FSImagePreTransactionalStorageInspector,
+    IOExceptions of stream closures can mask root exceptions.  (Uma Maheswara
+    Rao G via szetszwo)
+
 Release 0.23.0 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -750,6 +758,9 @@ Release 0.23.0 - Unreleased
 
   BUG FIXES
 
+    HDFS-2347. Fix checkpointTxnCount's comment about editlog size. 
+    (Uma Maheswara Rao G via mattf)
+
     HDFS-2011. Removal and restoration of storage directories on checkpointing
     failure doesn't work properly. (Ravi Prakash via mattf)
 

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -80,7 +80,9 @@ public class LayoutVersion {
     FEDERATION(-35, "Support for namenode federation"),
     LEASE_REASSIGNMENT(-36, "Support for persisting lease holder reassignment"),
     STORED_TXIDS(-37, "Transaction IDs are stored in edits log and image files"),
-    TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs");
+    TXID_BASED_LAYOUT(-38, "File names in NN Storage are based on transaction IDs"), 
+    EDITLOG_OP_OPTIMIZATION(-39,
+        "Use LongWritable and ShortWritable directly instead of ArrayWritable of UTF8");
     
     final int lv;
     final int ancestorLV;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java

@@ -60,7 +60,8 @@ class Checkpointer extends Daemon {
   private BackupNode backupNode;
   volatile boolean shouldRun;
   private long checkpointPeriod;    // in seconds
-  private long checkpointTxnCount;    // size (in MB) of current Edit Log
+  // Transactions count to trigger the checkpoint
+  private long checkpointTxnCount; 
 
   private String infoBindAddress;
 
@@ -108,7 +109,7 @@ class Checkpointer extends Daemon {
 
     LOG.info("Checkpoint Period : " + checkpointPeriod + " secs " +
              "(" + checkpointPeriod/60 + " min)");
-    LOG.info("Log Size Trigger  : " + checkpointTxnCount + " txns ");
+    LOG.info("Transactions count is  : " + checkpointTxnCount + ", to trigger checkpoint");
   }
 
   /**

+ 181 - 134
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -30,19 +30,16 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
-import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.util.PureJavaCrc32;
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes.*;
 import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ArrayWritable;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableFactories;
@@ -192,19 +189,17 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 nameReplicationPair[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(path), 
-        toLogReplication(replication),
-        toLogLong(mtime),
-        toLogLong(atime),
-        toLogLong(blockSize)};
-      new ArrayWritable(DeprecatedUTF8.class, nameReplicationPair).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeShort(replication, out);
+      FSImageSerialization.writeLong(mtime, out);
+      FSImageSerialization.writeLong(atime, out);
+      FSImageSerialization.writeLong(blockSize, out);
       new ArrayWritable(Block.class, blocks).write(out);
       permissions.write(out);
 
       if (this.opCode == OP_ADD) {
-        new DeprecatedUTF8(clientName).write(out);
-        new DeprecatedUTF8(clientMachine).write(out);
+        FSImageSerialization.writeString(clientName,out);
+        FSImageSerialization.writeString(clientMachine,out);
       }
     }
 
@@ -213,25 +208,43 @@ public abstract class FSEditLogOp {
         throws IOException {
       // versions > 0 support per file replication
       // get name and replication
-      this.length = in.readInt();
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+      }
       if (-7 == logVersion && length != 3||
           -17 < logVersion && logVersion < -7 && length != 4 ||
-          logVersion <= -17 && length != 5) {
+          (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
+              Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
         throw new IOException("Incorrect data format."  +
                               " logVersion is " + logVersion +
                               " but writables.length is " +
                               length + ". ");
       }
       this.path = FSImageSerialization.readString(in);
-      this.replication = readShort(in);
-      this.mtime = readLong(in);
+
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.replication = FSImageSerialization.readShort(in);
+        this.mtime = FSImageSerialization.readLong(in);
+      } else {
+        this.replication = readShort(in);
+        this.mtime = readLong(in);
+      }
+
       if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
-        this.atime = readLong(in);
+        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+          this.atime = FSImageSerialization.readLong(in);
+        } else {
+          this.atime = readLong(in);
+        }
       } else {
         this.atime = 0;
       }
       if (logVersion < -7) {
-        this.blockSize = readLong(in);
+        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+          this.blockSize = FSImageSerialization.readLong(in);
+        } else {
+          this.blockSize = readLong(in);
+        }
       } else {
         this.blockSize = 0;
       }
@@ -335,15 +348,19 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new DeprecatedUTF8(path).write(out);
-      new DeprecatedUTF8(Short.toString(replication)).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeShort(replication, out);
     }
     
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.path = FSImageSerialization.readString(in);
-      this.replication = readShort(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.replication = FSImageSerialization.readShort(in);
+      } else {
+        this.replication = readShort(in);
+      }
     }
   }
 
@@ -379,32 +396,45 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      int size = 1 + srcs.length + 1; // trg, srcs, timestamp
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[size];
+      FSImageSerialization.writeString(trg, out);
+            
+      DeprecatedUTF8 info[] = new DeprecatedUTF8[srcs.length];
       int idx = 0;
-      info[idx++] = new DeprecatedUTF8(trg);
       for(int i=0; i<srcs.length; i++) {
         info[idx++] = new DeprecatedUTF8(srcs[i]);
       }
-      info[idx] = toLogLong(timestamp);
       new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+
+      FSImageSerialization.writeLong(timestamp, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.length = in.readInt();
-      if (length < 3) { // trg, srcs.., timestam
-        throw new IOException("Incorrect data format. "
-                              + "Concat delete operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (length < 3) { // trg, srcs.., timestamp
+          throw new IOException("Incorrect data format. "
+              + "Concat delete operation.");
+        }
       }
       this.trg = FSImageSerialization.readString(in);
-      int srcSize = this.length - 1 - 1; //trg and timestamp
+      int srcSize = 0;
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        srcSize = in.readInt();
+      } else {
+        srcSize = this.length - 1 - 1; // trg and timestamp
+      }
       this.srcs = new String [srcSize];
       for(int i=0; i<srcSize;i++) {
         srcs[i]= FSImageSerialization.readString(in);
       }
-      this.timestamp = readLong(in);
+      
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
     }
   }
 
@@ -440,24 +470,28 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(src),
-        new DeprecatedUTF8(dst),
-        toLogLong(timestamp)};
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(src, out);
+      FSImageSerialization.writeString(dst, out);
+      FSImageSerialization.writeLong(timestamp, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.length = in.readInt();
-      if (this.length != 3) {
-        throw new IOException("Incorrect data format. "
-                              + "Old rename operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (this.length != 3) {
+          throw new IOException("Incorrect data format. "
+              + "Old rename operation.");
+        }
       }
       this.src = FSImageSerialization.readString(in);
       this.dst = FSImageSerialization.readString(in);
-      this.timestamp = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
     }
   }
 
@@ -487,22 +521,25 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(path),
-        toLogLong(timestamp)};
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeLong(timestamp, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.length = in.readInt();
-      if (this.length != 2) {
-        throw new IOException("Incorrect data format. "
-                              + "delete operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (this.length != 2) {
+          throw new IOException("Incorrect data format. " + "delete operation.");
+        }
       }
       this.path = FSImageSerialization.readString(in);
-      this.timestamp = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
     }
   }
 
@@ -538,12 +575,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] {
-        new DeprecatedUTF8(path),
-        toLogLong(timestamp), // mtime
-        toLogLong(timestamp) // atime, unused at this time
-      };
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeLong(timestamp, out); // mtime
+      FSImageSerialization.writeLong(timestamp, out); // atime, unused at this
       permissions.write(out);
     }
     
@@ -551,20 +585,32 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
 
-      this.length = in.readInt();
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+      }
       if (-17 < logVersion && length != 2 ||
-          logVersion <= -17 && length != 3) {
+          logVersion <= -17 && length != 3
+          && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         throw new IOException("Incorrect data format. "
                               + "Mkdir operation.");
       }
       this.path = FSImageSerialization.readString(in);
-      this.timestamp = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
 
       // The disk format stores atimes for directories as well.
       // However, currently this is not being updated/used because of
       // performance reasons.
       if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
-        /*unused this.atime = */readLong(in);
+        /* unused this.atime = */
+        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+          FSImageSerialization.readLong(in);
+        } else {
+          readLong(in);
+        }
       }
 
       if (logVersion <= -11) {
@@ -594,13 +640,13 @@ public abstract class FSEditLogOp {
     
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new LongWritable(genStamp).write(out);
+      FSImageSerialization.writeLong(genStamp, out);
     }
     
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.genStamp = in.readLong();
+      this.genStamp = FSImageSerialization.readLong(in);
     }
   }
 
@@ -678,7 +724,7 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new DeprecatedUTF8(src).write(out);
+      FSImageSerialization.writeString(src, out);
       permissions.write(out);
      }
  
@@ -721,11 +767,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 u = new DeprecatedUTF8(username == null? "": username);
-      DeprecatedUTF8 g = new DeprecatedUTF8(groupname == null? "": groupname);
-      new DeprecatedUTF8(src).write(out);
-      u.write(out);
-      g.write(out);
+      FSImageSerialization.writeString(src, out);
+      FSImageSerialization.writeString(username == null ? "" : username, out);
+      FSImageSerialization.writeString(groupname == null ? "" : groupname, out);
     }
 
     @Override
@@ -759,7 +803,7 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.src = FSImageSerialization.readString(in);
-      this.nsQuota = readLongWritable(in);
+      this.nsQuota = FSImageSerialization.readLong(in);
     }
   }
 
@@ -818,17 +862,17 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new DeprecatedUTF8(src).write(out);
-      new LongWritable(nsQuota).write(out);
-      new LongWritable(dsQuota).write(out);
+      FSImageSerialization.writeString(src, out);
+      FSImageSerialization.writeLong(nsQuota, out);
+      FSImageSerialization.writeLong(dsQuota, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.src = FSImageSerialization.readString(in);
-      this.nsQuota = readLongWritable(in);
-      this.dsQuota = readLongWritable(in);
+      this.nsQuota = FSImageSerialization.readLong(in);
+      this.dsQuota = FSImageSerialization.readLong(in);
     }
   }
 
@@ -864,24 +908,29 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(path),
-        toLogLong(mtime),
-        toLogLong(atime)};
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeLong(mtime, out);
+      FSImageSerialization.writeLong(atime, out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.length = in.readInt();
-      if (length != 3) {
-        throw new IOException("Incorrect data format. "
-                              + "times operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (length != 3) {
+          throw new IOException("Incorrect data format. " + "times operation.");
+        }
       }
       this.path = FSImageSerialization.readString(in);
-      this.mtime = readLong(in);
-      this.atime = readLong(in);
+
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.mtime = FSImageSerialization.readLong(in);
+        this.atime = FSImageSerialization.readLong(in);
+      } else {
+        this.mtime = readLong(in);
+        this.atime = readLong(in);
+      }
     }
   }
 
@@ -929,28 +978,33 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(path),
-        new DeprecatedUTF8(value),
-        toLogLong(mtime),
-        toLogLong(atime)};
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeString(value, out);
+      FSImageSerialization.writeLong(mtime, out);
+      FSImageSerialization.writeLong(atime, out);
       permissionStatus.write(out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-
-      this.length = in.readInt();
-      if (this.length != 4) {
-        throw new IOException("Incorrect data format. "
-                              + "symlink operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (this.length != 4) {
+          throw new IOException("Incorrect data format. "
+              + "symlink operation.");
+        }
       }
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
-      this.mtime = readLong(in);
-      this.atime = readLong(in);
+
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.mtime = FSImageSerialization.readLong(in);
+        this.atime = FSImageSerialization.readLong(in);
+      } else {
+        this.mtime = readLong(in);
+        this.atime = readLong(in);
+      }
       this.permissionStatus = PermissionStatus.read(in);
     }
   }
@@ -993,25 +1047,29 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      DeprecatedUTF8 info[] = new DeprecatedUTF8[] { 
-        new DeprecatedUTF8(src),
-        new DeprecatedUTF8(dst),
-        toLogLong(timestamp)};
-      new ArrayWritable(DeprecatedUTF8.class, info).write(out);
+      FSImageSerialization.writeString(src, out);
+      FSImageSerialization.writeString(dst, out);
+      FSImageSerialization.writeLong(timestamp, out);
       toBytesWritable(options).write(out);
     }
 
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      this.length = in.readInt();
-      if (this.length != 3) {
-        throw new IOException("Incorrect data format. "
-                              + "Rename operation.");
+      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.length = in.readInt();
+        if (this.length != 3) {
+          throw new IOException("Incorrect data format. " + "Rename operation.");
+        }
       }
       this.src = FSImageSerialization.readString(in);
       this.dst = FSImageSerialization.readString(in);
-      this.timestamp = readLong(in);
+
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.timestamp = FSImageSerialization.readLong(in);
+      } else {
+        this.timestamp = readLong(in);
+      }
       this.options = readRenameOptions(in);
     }
 
@@ -1068,9 +1126,9 @@ public abstract class FSEditLogOp {
 
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
-      new DeprecatedUTF8(leaseHolder).write(out);
-      new DeprecatedUTF8(path).write(out);
-      new DeprecatedUTF8(newHolder).write(out);
+      FSImageSerialization.writeString(leaseHolder, out);
+      FSImageSerialization.writeString(path, out);
+      FSImageSerialization.writeString(newHolder, out);
     }
 
     @Override
@@ -1109,7 +1167,7 @@ public abstract class FSEditLogOp {
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
-      toLogLong(expiryTime).write(out);
+      FSImageSerialization.writeLong(expiryTime, out);
     }
 
     @Override
@@ -1117,7 +1175,11 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       this.token.readFields(in);
-      this.expiryTime = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.expiryTime = FSImageSerialization.readLong(in);
+      } else {
+        this.expiryTime = readLong(in);
+      }
     }
   }
 
@@ -1148,7 +1210,7 @@ public abstract class FSEditLogOp {
     @Override 
     void writeFields(DataOutputStream out) throws IOException {
       token.write(out);
-      toLogLong(expiryTime).write(out);
+      FSImageSerialization.writeLong(expiryTime, out);
     }
 
     @Override
@@ -1156,7 +1218,11 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       this.token.readFields(in);
-      this.expiryTime = readLong(in);
+      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+        this.expiryTime = FSImageSerialization.readLong(in);
+      } else {
+        this.expiryTime = readLong(in);
+      }
     }
   }
 
@@ -1271,14 +1337,6 @@ public abstract class FSEditLogOp {
     return Long.parseLong(FSImageSerialization.readString(in));
   }
 
-  static private DeprecatedUTF8 toLogReplication(short replication) {
-    return new DeprecatedUTF8(Short.toString(replication));
-  }
-  
-  static private DeprecatedUTF8 toLogLong(long timestamp) {
-    return new DeprecatedUTF8(Long.toString(timestamp));
-  }
-
   /**
    * A class to read in blocks stored in the old format. The only two
    * fields in the block were blockid and length.
@@ -1314,17 +1372,6 @@ public abstract class FSEditLogOp {
     }
   }
 
-    // a place holder for reading a long
-  private static final LongWritable longWritable = new LongWritable();
-
-  /** Read an integer from an input stream */
-  private static long readLongWritable(DataInputStream in) throws IOException {
-    synchronized (longWritable) {
-      longWritable.readFields(in);
-      return longWritable.get();
-    }
-  }
-
   /**
    * Class for writing editlog ops
    */

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
+import org.apache.hadoop.io.IOUtils;
 
 /**
  * Inspects a FSImage storage directory in the "old" (pre-HDFS-1073) format.
@@ -130,8 +131,10 @@ class FSImagePreTransactionalStorageInspector extends FSImageStorageInspector {
       DataInputStream in = new DataInputStream(new FileInputStream(timeFile));
       try {
         timeStamp = in.readLong();
-      } finally {
         in.close();
+        in = null;
+      } finally {
+        IOUtils.cleanup(LOG, in);
       }
     }
     return timeStamp;

+ 32 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -36,6 +36,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 
@@ -72,6 +74,8 @@ public class FSImageSerialization {
    */
   static private final class TLData {
     final DeprecatedUTF8 U_STR = new DeprecatedUTF8();
+    final ShortWritable U_SHORT = new ShortWritable();
+    final LongWritable U_LONG = new LongWritable();
     final FsPermission FILE_PERM = new FsPermission((short) 0);
   }
 
@@ -215,7 +219,35 @@ public class FSImageSerialization {
     ustr.write(out);
   }
 
+  
+  /** read the long value */
+  static long readLong(DataInputStream in) throws IOException {
+    LongWritable ustr = TL_DATA.get().U_LONG;
+    ustr.readFields(in);
+    return ustr.get();
+  }
+
+  /** write the long value */
+  static void writeLong(long value, DataOutputStream out) throws IOException {
+    LongWritable uLong = TL_DATA.get().U_LONG;
+    uLong.set(value);
+    uLong.write(out);
+  }
 
+  /** read short value */
+  static short readShort(DataInputStream in) throws IOException {
+    ShortWritable uShort = TL_DATA.get().U_SHORT;
+    uShort.readFields(in);
+    return uShort.get();
+  }
+
+  /** write short value */
+  static void writeShort(short value, DataOutputStream out) throws IOException {
+    ShortWritable uShort = TL_DATA.get().U_SHORT;
+    uShort.set(value);
+    uShort.write(out);
+  }
+  
   // Same comments apply for this method as for readString()
   @SuppressWarnings("deprecation")
   public static byte[] readBytes(DataInputStream in) throws IOException {

+ 7 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -173,10 +173,12 @@ public class NNStorage extends Storage implements Closeable {
     try {
       oldFile.seek(0);
       int oldVersion = oldFile.readInt();
+      oldFile.close();
+      oldFile = null;
       if (oldVersion < LAST_PRE_UPGRADE_LAYOUT_VERSION)
         return false;
     } finally {
-      oldFile.close();
+      IOUtils.cleanup(LOG, oldFile);
     }
     return true;
   }
@@ -392,6 +394,8 @@ public class NNStorage extends Storage implements Closeable {
       BufferedReader br = new BufferedReader(new FileReader(txidFile));
       try {
         txid = Long.valueOf(br.readLine());
+        br.close();
+        br = null;
       } finally {
         IOUtils.cleanup(LOG, br);
       }
@@ -413,6 +417,8 @@ public class NNStorage extends Storage implements Closeable {
     try {
       fos.write(String.valueOf(txid).getBytes());
       fos.write('\n');
+      fos.close();
+      fos = null;
     } finally {
       IOUtils.cleanup(LOG, fos);
     }

+ 121 - 57
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/EditsLoaderCurrent.java

@@ -41,7 +41,7 @@ import static org.apache.hadoop.hdfs.tools.offlineEditsViewer.Tokenizer.VIntToke
 class EditsLoaderCurrent implements EditsLoader {
 
   private static int[] supportedVersions = { -18, -19, -20, -21, -22, -23, -24,
-      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
+      -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
 
   private EditsVisitor v;
   private int editsVersion = 0;
@@ -102,20 +102,29 @@ class EditsLoaderCurrent implements EditsLoader {
   private void visit_OP_ADD_or_OP_CLOSE(FSEditLogOpCodes editsOpCode)
     throws IOException {
     visitTxId();
-
-    IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
-    // this happens if the edits is not properly ended (-1 op code),
-    // it is padded at the end with all zeros, OP_ADD is zero so
-    // without this check we would treat all zeros as empty OP_ADD)
-    if(opAddLength.value == 0) {
-      throw new IOException("OpCode " + editsOpCode +
-        " has zero length (corrupted edits)");
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken opAddLength = v.visitInt(EditsElement.LENGTH);
+      // this happens if the edits is not properly ended (-1 op code),
+      // it is padded at the end with all zeros, OP_ADD is zero so
+      // without this check we would treat all zeros as empty OP_ADD)
+      if (opAddLength.value == 0) {
+        throw new IOException("OpCode " + editsOpCode
+            + " has zero length (corrupted edits)");
+      }
     }
+    
     v.visitStringUTF8(EditsElement.PATH);
-    v.visitStringUTF8(EditsElement.REPLICATION);
-    v.visitStringUTF8(EditsElement.MTIME);
-    v.visitStringUTF8(EditsElement.ATIME);
-    v.visitStringUTF8(EditsElement.BLOCKSIZE);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitShort(EditsElement.REPLICATION);
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+      v.visitLong(EditsElement.BLOCKSIZE);
+    } else {
+      v.visitStringUTF8(EditsElement.REPLICATION);
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+      v.visitStringUTF8(EditsElement.BLOCKSIZE);
+    }
     // now read blocks
     IntToken numBlocksToken = v.visitInt(EditsElement.NUMBLOCKS);
     for (int i = 0; i < numBlocksToken.value; i++) {
@@ -146,11 +155,16 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_RENAME_OLD() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.SOURCE);
     v.visitStringUTF8( EditsElement.DESTINATION);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -158,10 +172,15 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_DELETE() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -169,11 +188,17 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_MKDIR() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.TIMESTAMP);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
     // PERMISSION_STATUS
     v.visitEnclosingElement( EditsElement.PERMISSION_STATUS);
 
@@ -191,7 +216,11 @@ class EditsLoaderCurrent implements EditsLoader {
     visitTxId();
 
     v.visitStringUTF8(EditsElement.PATH);
-    v.visitStringUTF8(EditsElement.REPLICATION);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitShort(EditsElement.REPLICATION);
+    } else {
+      v.visitStringUTF8(EditsElement.REPLICATION);
+    }
   }
 
   /**
@@ -229,11 +258,17 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_TIMES() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.PATH);
-    v.visitStringUTF8( EditsElement.MTIME);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
   }
 
   /**
@@ -252,11 +287,16 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_RENAME() throws IOException {
     visitTxId();
-
-    v.visitInt(           EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8(    EditsElement.SOURCE);
     v.visitStringUTF8(    EditsElement.DESTINATION);
-    v.visitStringUTF8(    EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
     v.visitBytesWritable( EditsElement.RENAME_OPTIONS);
   }
 
@@ -265,15 +305,25 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_CONCAT_DELETE() throws IOException {
     visitTxId();
-
-    IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+    int sourceCount = 0;
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+      sourceCount = lengthToken.value - 2;
+    }
     v.visitStringUTF8(EditsElement.CONCAT_TARGET);
     // all except of CONCAT_TARGET and TIMESTAMP
-    int sourceCount = lengthToken.value - 2;
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      IntToken lengthToken = v.visitInt(EditsElement.LENGTH);
+      sourceCount = lengthToken.value;
+    }
     for(int i = 0; i < sourceCount; i++) {
       v.visitStringUTF8(EditsElement.CONCAT_SOURCE);
     }
-    v.visitStringUTF8(EditsElement.TIMESTAMP);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.TIMESTAMP);
+    } else {
+      v.visitStringUTF8(EditsElement.TIMESTAMP);
+    }
   }
 
   /**
@@ -281,12 +331,18 @@ class EditsLoaderCurrent implements EditsLoader {
    */
   private void visit_OP_SYMLINK() throws IOException {
     visitTxId();
-
-    v.visitInt(        EditsElement.LENGTH);
+    if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitInt(EditsElement.LENGTH);
+    }
     v.visitStringUTF8( EditsElement.SOURCE);
     v.visitStringUTF8( EditsElement.DESTINATION);
-    v.visitStringUTF8( EditsElement.MTIME);
-    v.visitStringUTF8( EditsElement.ATIME);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.MTIME);
+      v.visitLong(EditsElement.ATIME);
+    } else {
+      v.visitStringUTF8(EditsElement.MTIME);
+      v.visitStringUTF8(EditsElement.ATIME);
+    }
     // PERMISSION_STATUS
     v.visitEnclosingElement(EditsElement.PERMISSION_STATUS);
 
@@ -303,15 +359,19 @@ class EditsLoaderCurrent implements EditsLoader {
   private void visit_OP_GET_DELEGATION_TOKEN() throws IOException {
     visitTxId();
     
-      v.visitByte(       EditsElement.T_VERSION);
-      v.visitStringText( EditsElement.T_OWNER);
-      v.visitStringText( EditsElement.T_RENEWER);
-      v.visitStringText( EditsElement.T_REAL_USER);
-      v.visitVLong(      EditsElement.T_ISSUE_DATE);
-      v.visitVLong(      EditsElement.T_MAX_DATE);
-      v.visitVInt(       EditsElement.T_SEQUENCE_NUMBER);
-      v.visitVInt(       EditsElement.T_MASTER_KEY_ID);
-      v.visitStringUTF8( EditsElement.T_EXPIRY_TIME);
+    v.visitByte(EditsElement.T_VERSION);
+    v.visitStringText(EditsElement.T_OWNER);
+    v.visitStringText(EditsElement.T_RENEWER);
+    v.visitStringText(EditsElement.T_REAL_USER);
+    v.visitVLong(EditsElement.T_ISSUE_DATE);
+    v.visitVLong(EditsElement.T_MAX_DATE);
+    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
+    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.T_EXPIRY_TIME);
+    } else {
+      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
+    }
   }
 
   /**
@@ -321,15 +381,19 @@ class EditsLoaderCurrent implements EditsLoader {
     throws IOException {
     visitTxId();
 
-      v.visitByte(       EditsElement.T_VERSION);
-      v.visitStringText( EditsElement.T_OWNER);
-      v.visitStringText( EditsElement.T_RENEWER);
-      v.visitStringText( EditsElement.T_REAL_USER);
-      v.visitVLong(      EditsElement.T_ISSUE_DATE);
-      v.visitVLong(      EditsElement.T_MAX_DATE);
-      v.visitVInt(       EditsElement.T_SEQUENCE_NUMBER);
-      v.visitVInt(       EditsElement.T_MASTER_KEY_ID);
-      v.visitStringUTF8( EditsElement.T_EXPIRY_TIME);
+    v.visitByte(EditsElement.T_VERSION);
+    v.visitStringText(EditsElement.T_OWNER);
+    v.visitStringText(EditsElement.T_RENEWER);
+    v.visitStringText(EditsElement.T_REAL_USER);
+    v.visitVLong(EditsElement.T_ISSUE_DATE);
+    v.visitVLong(EditsElement.T_MAX_DATE);
+    v.visitVInt(EditsElement.T_SEQUENCE_NUMBER);
+    v.visitVInt(EditsElement.T_MASTER_KEY_ID);
+    if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, editsVersion)) {
+      v.visitLong(EditsElement.T_EXPIRY_TIME);
+    } else {
+      v.visitStringUTF8(EditsElement.T_EXPIRY_TIME);
+    }
   }
 
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -122,7 +122,7 @@ class ImageLoaderCurrent implements ImageLoader {
   protected final DateFormat dateFormat = 
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
-      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38};
+      -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39};
   private int imageVersion = 0;
 
   /* (non-Javadoc)

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

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.*;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.FileInputStream;
@@ -29,7 +32,6 @@ import java.util.SortedMap;
 
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;

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

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Before;

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

@@ -17,6 +17,9 @@ Trunk (unreleased changes)
     MAPREDUCE-2934. MR portion of HADOOP-7607 - Simplify the RPC proxy cleanup
                     process (atm)
 
+    MAPREDUCE-2836. Provide option to fail jobs when submitted to non-existent
+    fair scheduler pools. (Ahmed Radwan via todd)
+
   BUG FIXES
 
     MAPREDUCE-2784. [Gridmix] Bug fixes in ExecutionSummarizer and 
@@ -298,6 +301,12 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2672. MR-279: JobHistory Server needs Analysis this job. 
     (Robert Evans via mahadev)
 
+    MAPREDUCE-2965. Streamlined the methods hashCode(), equals(), compareTo()
+    and toString() for all IDs. (Siddharth Seth via vinodkv)
+
+    MAPREDUCE-2726. Added job-file to the AM and JobHistoryServer web
+    interfaces. (Jeffrey Naisbitt via vinodkv)
+
   OPTIMIZATIONS
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
@@ -1333,6 +1342,34 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-3007. Fixed Yarn Mapreduce client to be able to connect to 
     JobHistoryServer in secure mode. (vinodkv)
 
+    MAPREDUCE-2987. Fixed display of logged user on RM Web-UI. (Thomas Graves
+    via acmurthy)
+
+    MAPREDUCE-3006. Fixed MapReduce AM to exit only after properly writing out
+    history file. (vinodkv)
+
+    MAPREDUCE-2925. Fixed Yarn+MR client code to behave saner with completed
+    jobs. (Devaraj K via vinodkv)
+
+    MAPREDUCE-3030. Fixed a bug in NodeId.equals() that was causing RM to
+    reject all NMs. (Devaraj K via vinodkv)
+
+    MAPREDUCE-3042. Fixed default ResourceTracker address. (Chris Riccomini
+    via acmurthy) 
+
+    MAPREDUCE-3038. job history server not starting because conf() missing
+    HsController (Jeffrey Naisbitt via mahadev)
+
+    MAPREDUCE-3004. Fix ReduceTask to not assume 'local' mode in YARN. (Hitesh
+    Shah via acmurthy)
+
+    MAPREDUCE-3017. The Web UI shows FINISHED for killed/successful/failed jobs.
+    (mahadev)
+
+    MAPREDUCE-3040. Fixed extra copy of Configuration in
+    YarnClientProtocolProvider and ensured MiniMRYarnCluster sets JobHistory
+    configuration for tests. (acmurthy) 
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -1346,6 +1383,9 @@ Release 0.22.0 - Unreleased
     MAPREDUCE-2994. Fixed a bug in ApplicationID parsing that affects RM
     UI. (Devaraj K via vinodkv)
 
+    MAPREDUCE-1788. o.a.h.mapreduce.Job shouldn't make a copy of the JobConf.
+    (Arun Murthy via mahadev)
+
   NEW FEATURES
 
     MAPREDUCE-1804. Stress-test tool for HDFS introduced in HDFS-708.

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java

@@ -74,7 +74,7 @@ public class JobHistoryEventHandler extends AbstractService
 
   private BlockingQueue<JobHistoryEvent> eventQueue =
     new LinkedBlockingQueue<JobHistoryEvent>();
-  private Thread eventHandlingThread;
+  protected Thread eventHandlingThread;
   private volatile boolean stopped;
   private final Object lock = new Object();
 

+ 171 - 81
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -56,12 +56,14 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher;
+import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl;
 import org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.metrics.MRAppMetrics;
 import org.apache.hadoop.mapreduce.v2.app.recover.Recovery;
 import org.apache.hadoop.mapreduce.v2.app.recover.RecoveryService;
 import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocator;
+import org.apache.hadoop.mapreduce.v2.app.rm.ContainerAllocatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.DefaultSpeculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
@@ -83,6 +85,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.service.Service;
 
@@ -126,6 +129,7 @@ public class MRAppMaster extends CompositeService {
   private TaskAttemptListener taskAttemptListener;
   private JobTokenSecretManager jobTokenSecretManager =
       new JobTokenSecretManager();
+  private JobEventDispatcher jobEventDispatcher;
 
   private Job job;
   
@@ -148,7 +152,7 @@ public class MRAppMaster extends CompositeService {
 
   @Override
   public void init(final Configuration conf) {
-    context = new RunningAppContext();
+    context = new RunningAppContext(conf);
 
     // Job name is the same as the app name util we support DAG of jobs
     // for an app later
@@ -182,18 +186,17 @@ public class MRAppMaster extends CompositeService {
     //service to log job history events
     EventHandler<JobHistoryEvent> historyService = 
         createJobHistoryHandler(context);
-    addIfService(historyService);
+    dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
+        historyService);
 
-    JobEventDispatcher synchronousJobEventDispatcher = new JobEventDispatcher();
+    this.jobEventDispatcher = new JobEventDispatcher();
 
     //register the event dispatchers
-    dispatcher.register(JobEventType.class, synchronousJobEventDispatcher);
+    dispatcher.register(JobEventType.class, jobEventDispatcher);
     dispatcher.register(TaskEventType.class, new TaskEventDispatcher());
     dispatcher.register(TaskAttemptEventType.class, 
         new TaskAttemptEventDispatcher());
     dispatcher.register(TaskCleaner.EventType.class, taskCleaner);
-    dispatcher.register(org.apache.hadoop.mapreduce.jobhistory.EventType.class,
-        historyService);
     
     if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
         || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
@@ -203,10 +206,34 @@ public class MRAppMaster extends CompositeService {
     }
 
     dispatcher.register(Speculator.EventType.class,
-        new SpeculatorEventDispatcher());
+        new SpeculatorEventDispatcher(conf));
 
-    Credentials fsTokens = new Credentials();
+    // service to allocate containers from RM (if non-uber) or to fake it (uber)
+    containerAllocator = createContainerAllocator(clientService, context);
+    addIfService(containerAllocator);
+    dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
+
+    // corresponding service to launch allocated containers via NodeManager
+    containerLauncher = createContainerLauncher(context);
+    addIfService(containerLauncher);
+    dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
+
+    // Add the JobHistoryEventHandler last so that it is properly stopped first.
+    // This will guarantee that all history-events are flushed before AM goes
+    // ahead with shutdown.
+    // Note: Even though JobHistoryEventHandler is started last, if any
+    // component creates a JobHistoryEvent in the meanwhile, it will be just be
+    // queued inside the JobHistoryEventHandler 
+    addIfService(historyService);
 
+    super.init(conf);
+  } // end of init()
+
+  /** Create and initialize (but don't start) a single job. */
+  protected Job createJob(Configuration conf) {
+
+    // ////////// Obtain the tokens needed by the job. //////////
+    Credentials fsTokens = new Credentials();
     UserGroupInformation currentUser = null;
 
     try {
@@ -234,66 +261,12 @@ public class MRAppMaster extends CompositeService {
     } catch (IOException e) {
       throw new YarnException(e);
     }
-
-    super.init(conf);
-
-    //---- start of what used to be startJobs() code:
-
-    Configuration config = getConfig();
-
-    job = createJob(config, fsTokens, currentUser.getUserName());
-
-    /** create a job event for job intialization */
-    JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
-    /** send init to the job (this does NOT trigger job execution) */
-    synchronousJobEventDispatcher.handle(initJobEvent);
-
-    // send init to speculator. This won't yest start as dispatcher isn't
-    // started yet.
-    dispatcher.getEventHandler().handle(
-        new SpeculatorEvent(job.getID(), clock.getTime()));
-
-    // JobImpl's InitTransition is done (call above is synchronous), so the
-    // "uber-decision" (MR-1220) has been made.  Query job and switch to
-    // ubermode if appropriate (by registering different container-allocator
-    // and container-launcher services/event-handlers).
-
-    if (job.isUber()) {
-      LOG.info("MRAppMaster uberizing job " + job.getID()
-               + " in local container (\"uber-AM\").");
-    } else {
-      LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
-               + "job " + job.getID() + ".");
-    }
-
-    // service to allocate containers from RM (if non-uber) or to fake it (uber)
-    containerAllocator =
-        createContainerAllocator(clientService, context, job.isUber());
-    addIfService(containerAllocator);
-    dispatcher.register(ContainerAllocator.EventType.class, containerAllocator);
-    if (containerAllocator instanceof Service) {
-      ((Service) containerAllocator).init(config);
-    }
-
-    // corresponding service to launch allocated containers via NodeManager
-    containerLauncher = createContainerLauncher(context, job.isUber());
-    addIfService(containerLauncher);
-    dispatcher.register(ContainerLauncher.EventType.class, containerLauncher);
-    if (containerLauncher instanceof Service) {
-      ((Service) containerLauncher).init(config);
-    }
-
-  } // end of init()
-
-  /** Create and initialize (but don't start) a single job. 
-   * @param fsTokens */
-  protected Job createJob(Configuration conf, Credentials fsTokens, 
-      String user) {
+    // ////////// End of obtaining the tokens needed by the job. //////////
 
     // create single job
     Job newJob = new JobImpl(appID, conf, dispatcher.getEventHandler(),
         taskAttemptListener, jobTokenSecretManager, fsTokens, clock, startCount,
-        completedTasksFromPreviousRun, metrics, user);
+        completedTasksFromPreviousRun, metrics, currentUser.getUserName());
     ((RunningAppContext) context).jobs.put(newJob.getID(), newJob);
 
     dispatcher.register(JobFinishEvent.Type.class,
@@ -388,19 +361,13 @@ public class MRAppMaster extends CompositeService {
   }
 
   protected ContainerAllocator createContainerAllocator(
-      ClientService clientService, AppContext context, boolean isLocal) {
-    //return new StaticContainerAllocator(context);
-    return isLocal
-        ? new LocalContainerAllocator(clientService, context)
-        : new RMContainerAllocator(clientService, context);
+      final ClientService clientService, final AppContext context) {
+    return new ContainerAllocatorRouter(clientService, context);
   }
 
-  protected ContainerLauncher createContainerLauncher(AppContext context,
-                                                      boolean isLocal) {
-    return isLocal
-        ? new LocalContainerLauncher(context,
-            (TaskUmbilicalProtocol) taskAttemptListener)
-        : new ContainerLauncherImpl(context);
+  protected ContainerLauncher
+      createContainerLauncher(final AppContext context) {
+    return new ContainerLauncherRouter(context);
   }
 
   //TODO:should have an interface for MRClientService
@@ -440,9 +407,96 @@ public class MRAppMaster extends CompositeService {
     return taskAttemptListener;
   }
 
-  class RunningAppContext implements AppContext {
+  /**
+   * By the time life-cycle of this router starts, job-init would have already
+   * happened.
+   */
+  private final class ContainerAllocatorRouter extends AbstractService
+      implements ContainerAllocator {
+    private final ClientService clientService;
+    private final AppContext context;
+    private ContainerAllocator containerAllocator;
+
+    ContainerAllocatorRouter(ClientService clientService,
+        AppContext context) {
+      super(ContainerAllocatorRouter.class.getName());
+      this.clientService = clientService;
+      this.context = context;
+    }
+
+    @Override
+    public synchronized void start() {
+      if (job.isUber()) {
+        this.containerAllocator = new LocalContainerAllocator(
+            this.clientService, this.context);
+      } else {
+        this.containerAllocator = new RMContainerAllocator(
+            this.clientService, this.context);
+      }
+      ((Service)this.containerAllocator).init(getConfig());
+      ((Service)this.containerAllocator).start();
+      super.start();
+    }
+
+    @Override
+    public synchronized void stop() {
+      ((Service)this.containerAllocator).stop();
+      super.stop();
+    }
+
+    @Override
+    public void handle(ContainerAllocatorEvent event) {
+      this.containerAllocator.handle(event);
+    }
+  }
+
+  /**
+   * By the time life-cycle of this router starts, job-init would have already
+   * happened.
+   */
+  private final class ContainerLauncherRouter extends AbstractService
+      implements ContainerLauncher {
+    private final AppContext context;
+    private ContainerLauncher containerLauncher;
+
+    ContainerLauncherRouter(AppContext context) {
+      super(ContainerLauncherRouter.class.getName());
+      this.context = context;
+    }
+
+    @Override
+    public synchronized void start() {
+      if (job.isUber()) {
+        this.containerLauncher = new LocalContainerLauncher(context,
+            (TaskUmbilicalProtocol) taskAttemptListener);
+      } else {
+        this.containerLauncher = new ContainerLauncherImpl(context);
+      }
+      ((Service)this.containerLauncher).init(getConfig());
+      ((Service)this.containerLauncher).start();
+      super.start();
+    }
+
+    @Override
+    public void handle(ContainerLauncherEvent event) {
+        this.containerLauncher.handle(event);
+    }
+
+    @Override
+    public synchronized void stop() {
+      ((Service)this.containerLauncher).stop();
+      super.stop();
+    }
+  }
+
+  private class RunningAppContext implements AppContext {
 
-    private Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
+    private final Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
+    private final Configuration conf;
+
+    public RunningAppContext(Configuration config) {
+      this.conf = config;
+    }
 
     @Override
     public ApplicationAttemptId getApplicationAttemptId() {
@@ -481,7 +535,7 @@ public class MRAppMaster extends CompositeService {
 
     @Override
     public CharSequence getUser() {
-      return getConfig().get(MRJobConfig.USER_NAME);
+      return this.conf.get(MRJobConfig.USER_NAME);
     }
 
     @Override
@@ -492,13 +546,45 @@ public class MRAppMaster extends CompositeService {
 
   @Override
   public void start() {
+
+    ///////////////////// Create the job itself.
+    job = createJob(getConfig());
+    // End of creating the job.
+
     // metrics system init is really init & start.
     // It's more test friendly to put it here.
     DefaultMetricsSystem.initialize("MRAppMaster");
 
-    startJobs();
+    /** create a job event for job intialization */
+    JobEvent initJobEvent = new JobEvent(job.getID(), JobEventType.JOB_INIT);
+    /** send init to the job (this does NOT trigger job execution) */
+    // This is a synchronous call, not an event through dispatcher. We want
+    // job-init to be done completely here.
+    jobEventDispatcher.handle(initJobEvent);
+
+    // send init to speculator. This won't yest start as dispatcher isn't
+    // started yet.
+    dispatcher.getEventHandler().handle(
+        new SpeculatorEvent(job.getID(), clock.getTime()));
+
+    // JobImpl's InitTransition is done (call above is synchronous), so the
+    // "uber-decision" (MR-1220) has been made.  Query job and switch to
+    // ubermode if appropriate (by registering different container-allocator
+    // and container-launcher services/event-handlers).
+
+    if (job.isUber()) {
+      LOG.info("MRAppMaster uberizing job " + job.getID()
+               + " in local container (\"uber-AM\").");
+    } else {
+      LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
+               + "job " + job.getID() + ".");
+    }
+
     //start all the components
     super.start();
+
+    // All components have started, start the job.
+    startJobs();
   }
 
   /**
@@ -546,10 +632,14 @@ public class MRAppMaster extends CompositeService {
 
   private class SpeculatorEventDispatcher implements
       EventHandler<SpeculatorEvent> {
+    private final Configuration conf;
+    public SpeculatorEventDispatcher(Configuration config) {
+      this.conf = config;
+    }
     @Override
     public void handle(SpeculatorEvent event) {
-      if (getConfig().getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
-          || getConfig().getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
+      if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
+          || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
         // Speculator IS enabled, direct the event to there.
         speculator.handle(event);
       }

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java

@@ -32,6 +32,7 @@ public class AMWebApp extends WebApp implements AMParams {
     route("/", AppController.class);
     route("/app", AppController.class);
     route(pajoin("/job", JOB_ID), AppController.class, "job");
+    route(pajoin("/conf", JOB_ID), AppController.class, "conf");
     route(pajoin("/jobcounters", JOB_ID), AppController.class, "jobCounters");
     route(pajoin("/tasks", JOB_ID, TASK_TYPE), AppController.class, "tasks");
     route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),

+ 15 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java

@@ -185,6 +185,21 @@ public class AppController extends Controller implements AMParams {
     }
   }
 
+  /**
+   * @return the page that will be used to render the /conf page
+   */
+  protected Class<? extends View> confPage() {
+    return JobConfPage.class;
+  }
+
+  /**
+   * Render the /conf page
+   */
+  public void conf() {
+    requireJob();
+    render(confPage());
+  }
+
   /**
    * Render a BAD_REQUEST error.
    * @param s the error message to include.

+ 96 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobConfPage.java

@@ -0,0 +1,96 @@
+/**
+* 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.mapreduce.v2.app.webapp;
+
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.postInitID;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
+
+import org.apache.hadoop.mapreduce.v2.app.webapp.ConfBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+/**
+ * Render a page with the configuration for a given job in it.
+ */
+public class JobConfPage extends AppView {
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsView#preHead(org.apache.hadoop.yarn.webapp.hamlet.Hamlet.HTML)
+   */
+  @Override protected void preHead(Page.HTML<_> html) {
+    String jobID = $(JOB_ID);
+    set(TITLE, jobID.isEmpty() ? "Bad request: missing job ID"
+        : join("Configuration for MapReduce Job ", $(JOB_ID)));
+    commonPreHead(html);
+    set(DATATABLES_ID, "conf");
+    set(initID(DATATABLES, "conf"), confTableInit());
+    set(postInitID(DATATABLES, "conf"), confPostTableInit());
+    setTableStyles(html, "conf");
+  }
+
+  /**
+   * The body of this block is the configuration block.
+   * @return ConfBlock.class
+   */
+  @Override protected Class<? extends SubView> content() {
+    return ConfBlock.class;
+  }
+
+  /**
+   * @return the end of the JS map that is the jquery datatable config for the
+   * conf table.
+   */
+  private String confTableInit() {
+    return tableInit().append("}").toString();
+  }
+
+  /**
+   * @return the java script code to allow the jquery conf datatable to filter
+   * by column.
+   */
+  private String confPostTableInit() {
+    return "var confInitVals = new Array();\n" +
+    "$('tfoot input').keyup( function () \n{"+
+    "  confDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
+    "} );\n"+
+    "$('tfoot input').each( function (i) {\n"+
+    "  confInitVals[i] = this.value;\n"+
+    "} );\n"+
+    "$('tfoot input').focus( function () {\n"+
+    "  if ( this.className == 'search_init' )\n"+
+    "  {\n"+
+    "    this.className = '';\n"+
+    "    this.value = '';\n"+
+    "  }\n"+
+    "} );\n"+
+    "$('tfoot input').blur( function (i) {\n"+
+    "  if ( this.value == '' )\n"+
+    "  {\n"+
+    "    this.className = 'search_init';\n"+
+    "    this.value = confInitVals[$('tfoot input').index(this)];\n"+
+    "  }\n"+
+    "} );\n";
+  }
+}

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/NavBlock.java

@@ -52,6 +52,7 @@ public class NavBlock extends HtmlBlock {
         ul().
           li().a(url("job", jobid), "Overview")._().
           li().a(url("jobcounters", jobid), "Counters")._().
+          li().a(url("conf", jobid), "Configuration")._().
           li().a(url("tasks", jobid, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce tasks")._()._();
     }

+ 20 - 16
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java

@@ -63,6 +63,7 @@ import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleaner;
 import org.apache.hadoop.mapreduce.v2.app.taskclean.TaskCleanupEvent;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -234,11 +235,16 @@ public class MRApp extends MRAppMaster {
   }
 
   @Override
-  protected Job createJob(Configuration conf, Credentials fsTokens,
-      String user) {
-    Job newJob = new TestJob(getAppID(), getDispatcher().getEventHandler(),
+  protected Job createJob(Configuration conf) {
+    UserGroupInformation currentUser = null;
+    try {
+      currentUser = UserGroupInformation.getCurrentUser();
+    } catch (IOException e) {
+      throw new YarnException(e);
+    }
+    Job newJob = new TestJob(conf, getAppID(), getDispatcher().getEventHandler(),
                              getTaskAttemptListener(), getContext().getClock(),
-                             user);
+                             currentUser.getUserName());
     ((AppContext) getContext()).getAllJobs().put(newJob.getID(), newJob);
 
     getDispatcher().register(JobFinishEvent.Type.class,
@@ -279,8 +285,7 @@ public class MRApp extends MRAppMaster {
   }
   
   @Override
-  protected ContainerLauncher createContainerLauncher(AppContext context,
-                                                      boolean isLocal) {
+  protected ContainerLauncher createContainerLauncher(AppContext context) {
     return new MockContainerLauncher();
   }
 
@@ -317,7 +322,7 @@ public class MRApp extends MRAppMaster {
 
   @Override
   protected ContainerAllocator createContainerAllocator(
-      ClientService clientService, AppContext context, boolean isLocal) {
+      ClientService clientService, AppContext context) {
     return new ContainerAllocator(){
       private int containerCount;
       @Override
@@ -369,12 +374,14 @@ public class MRApp extends MRAppMaster {
 
   class TestJob extends JobImpl {
     //override the init transition
+    private final TestInitTransition initTransition = new TestInitTransition(
+        maps, reduces);
     StateMachineFactory<JobImpl, JobState, JobEventType, JobEvent> localFactory
         = stateMachineFactory.addTransition(JobState.NEW,
             EnumSet.of(JobState.INITED, JobState.FAILED),
             JobEventType.JOB_INIT,
             // This is abusive.
-            new TestInitTransition(getConfig(), maps, reduces));
+            initTransition);
 
     private final StateMachine<JobState, JobEventType, JobEvent>
         localStateMachine;
@@ -384,10 +391,10 @@ public class MRApp extends MRAppMaster {
       return localStateMachine;
     }
 
-    public TestJob(ApplicationId appID, EventHandler eventHandler,
-        TaskAttemptListener taskAttemptListener, Clock clock, 
-        String user) {
-      super(appID, new Configuration(), eventHandler, taskAttemptListener,
+    public TestJob(Configuration conf, ApplicationId appID,
+        EventHandler eventHandler, TaskAttemptListener taskAttemptListener,
+        Clock clock, String user) {
+      super(appID, conf, eventHandler, taskAttemptListener,
           new JobTokenSecretManager(), new Credentials(), clock, getStartCount(), 
           getCompletedTaskFromPreviousRun(), metrics, user);
 
@@ -399,17 +406,14 @@ public class MRApp extends MRAppMaster {
 
   //Override InitTransition to not look for split files etc
   static class TestInitTransition extends JobImpl.InitTransition {
-    private Configuration config;
     private int maps;
     private int reduces;
-    TestInitTransition(Configuration config, int maps, int reduces) {
-      this.config = config;
+    TestInitTransition(int maps, int reduces) {
       this.maps = maps;
       this.reduces = reduces;
     }
     @Override
     protected void setup(JobImpl job) throws IOException {
-      job.conf = config;
       job.conf.setInt(MRJobConfig.NUM_REDUCES, reduces);
       job.remoteJobConfFile = new Path("test");
     }

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java

@@ -94,7 +94,7 @@ public class MRAppBenchmark {
     
     @Override
     protected ContainerAllocator createContainerAllocator(
-        ClientService clientService, AppContext context, boolean isLocal) {
+        ClientService clientService, AppContext context) {
       return new ThrottledContainerAllocator();
     }
     

+ 6 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java

@@ -169,7 +169,7 @@ public class TestFail {
 
   @Test
   public void testTaskFailWithUnusedContainer() throws Exception {
-    MRApp app = new FailingTaskWithUnusedContainer();
+    MRApp app = new MRAppWithFailingTaskAndUnusedContainer();
     Configuration conf = new Configuration();
     int maxAttempts = 1;
     conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, maxAttempts);
@@ -194,21 +194,21 @@ public class TestFail {
     app.waitForState(job, JobState.FAILED);
   }
 
-  static class FailingTaskWithUnusedContainer extends MRApp {
+  static class MRAppWithFailingTaskAndUnusedContainer extends MRApp {
 
-    public FailingTaskWithUnusedContainer() {
+    public MRAppWithFailingTaskAndUnusedContainer() {
       super(1, 0, false, "TaskFailWithUnsedContainer", true);
     }
 
-    protected ContainerLauncher createContainerLauncher(AppContext context,
-        boolean isLocal) {
+    @Override
+    protected ContainerLauncher createContainerLauncher(AppContext context) {
       return new ContainerLauncherImpl(context) {
         @Override
         public void handle(ContainerLauncherEvent event) {
 
           switch (event.getType()) {
           case CONTAINER_REMOTE_LAUNCH:
-            super.handle(event);
+            super.handle(event); // Unused event and container.
             break;
           case CONTAINER_REMOTE_CLEANUP:
             getContext().getEventHandler().handle(

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

@@ -24,10 +24,10 @@ import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
-import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
@@ -195,6 +195,7 @@ public class TestMRApp {
   public static void main(String[] args) throws Exception {
     TestMRApp t = new TestMRApp();
     t.testMapReduce();
+    t.testZeroMapReduces();
     t.testCommitPending();
     t.testCompletedMapsForReduceSlowstart();
     t.testJobError();

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java

@@ -158,6 +158,11 @@ public class TestAMWebApp {
     return params;
   }
 
+  @Test public void testConfView() {
+    WebAppTests.testPage(JobConfPage.class, AppContext.class,
+                         new TestAppContext());
+  }
+
   public static void main(String[] args) {
     WebApps.$for("yarn", AppContext.class, new TestAppContext(0, 8, 88, 4)).
         at(58888).inDevMode().start(new AMWebApp()).joinThread();

+ 84 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobId.java

@@ -18,13 +18,95 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records;
 
+import java.text.NumberFormat;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 
-public interface JobId {
+/**
+ * <p><code>JobId</code> represents the <em>globally unique</em> 
+ * identifier for a MapReduce job.</p>
+ * 
+ * <p>The globally unique nature of the identifier is achieved by using the 
+ * <em>cluster timestamp</em> from the associated ApplicationId. i.e. 
+ * start-time of the <code>ResourceManager</code> along with a monotonically
+ * increasing counter for the jobId.</p>
+ */
+public abstract class JobId implements Comparable<JobId> {
+
+  /**
+   * Get the associated <em>ApplicationId</em> which represents the 
+   * start time of the <code>ResourceManager</code> and is used to generate 
+   * the globally unique <code>JobId</code>.
+   * @return associated <code>ApplicationId</code>
+   */
   public abstract ApplicationId getAppId();
+  
+  /**
+   * Get the short integer identifier of the <code>JobId</code>
+   * which is unique for all applications started by a particular instance
+   * of the <code>ResourceManager</code>.
+   * @return short integer identifier of the <code>JobId</code>
+   */
   public abstract int getId();
   
   public abstract void setAppId(ApplicationId appId);
   public abstract void setId(int id);
 
-}
+
+  protected static final String JOB = "job";
+  protected static final char SEPARATOR = '_';
+  static final ThreadLocal<NumberFormat> jobIdFormat =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder(JOB);
+    builder.append(SEPARATOR);
+    builder.append(getAppId().getClusterTimestamp());
+    builder.append(SEPARATOR);
+    builder.append(jobIdFormat.get().format(getId()));
+    return builder.toString();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + getAppId().hashCode();
+    result = prime * result + getId();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    JobId other = (JobId) obj;
+    if (!this.getAppId().equals(other.getAppId()))
+      return false;
+    if (this.getId() != other.getId())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(JobId other) {
+    int appIdComp = this.getAppId().compareTo(other.getAppId());
+    if (appIdComp == 0) {
+      return this.getId() - other.getId();
+    } else {
+      return appIdComp;
+    }
+  }
+}

+ 80 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptId.java

@@ -18,10 +18,87 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records;
 
-public interface TaskAttemptId {
+/**
+ * <p>
+ * <code>TaskAttemptId</code> represents the unique identifier for a task
+ * attempt. Each task attempt is one particular instance of a Map or Reduce Task
+ * identified by its TaskId.
+ * </p>
+ * 
+ * <p>
+ * TaskAttemptId consists of 2 parts. First part is the <code>TaskId</code>,
+ * that this <code>TaskAttemptId</code> belongs to. Second part is the task
+ * attempt number.
+ * </p>
+ */
+public abstract class TaskAttemptId implements Comparable<TaskAttemptId> {
+  /**
+   * @return the associated TaskId.
+   */
   public abstract TaskId getTaskId();
+
+  /**
+   * @return the attempt id.
+   */
   public abstract int getId();
-  
+
   public abstract void setTaskId(TaskId taskId);
+
   public abstract void setId(int id);
-}
+
+  protected static final String TASKATTEMPT = "attempt";
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + getId();
+    result =
+        prime * result + ((getTaskId() == null) ? 0 : getTaskId().hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    TaskAttemptId other = (TaskAttemptId) obj;
+    if (getId() != other.getId())
+      return false;
+    if (!getTaskId().equals(other.getTaskId()))
+      return false;
+    return true;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder(TASKATTEMPT);
+    TaskId taskId = getTaskId();
+    builder.append("_").append(
+        taskId.getJobId().getAppId().getClusterTimestamp());
+    builder.append("_").append(
+        JobId.jobIdFormat.get().format(
+            getTaskId().getJobId().getAppId().getId()));
+    builder.append("_");
+    builder.append(taskId.getTaskType() == TaskType.MAP ? "m" : "r");
+    builder.append("_")
+        .append(TaskId.taskIdFormat.get().format(taskId.getId()));
+    builder.append("_");
+    builder.append(getId());
+    return builder.toString();
+  }
+
+  @Override
+  public int compareTo(TaskAttemptId other) {
+    int taskIdComp = this.getTaskId().compareTo(other.getTaskId());
+    if (taskIdComp == 0) {
+      return this.getId() - other.getId();
+    } else {
+      return taskIdComp;
+    }
+  }
+}

+ 101 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskId.java

@@ -18,12 +18,109 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records;
 
-public interface TaskId {
+import java.text.NumberFormat;
+
+/**
+ * <p>
+ * <code>TaskId</code> represents the unique identifier for a Map or Reduce
+ * Task.
+ * </p>
+ * 
+ * <p>
+ * TaskId consists of 3 parts. First part is <code>JobId</code>, that this Task
+ * belongs to. Second part of the TaskId is either 'm' or 'r' representing
+ * whether the task is a map task or a reduce task. And the third part is the
+ * task number.
+ * </p>
+ */
+public abstract class TaskId implements Comparable<TaskId> {
+
+  /**
+   * @return the associated <code>JobId</code>
+   */
   public abstract JobId getJobId();
-  public abstract  TaskType getTaskType();
+
+  /**
+   * @return the type of the task - MAP/REDUCE
+   */
+  public abstract TaskType getTaskType();
+
+  /**
+   * @return the task number.
+   */
   public abstract int getId();
-  
+
   public abstract void setJobId(JobId jobId);
+
   public abstract void setTaskType(TaskType taskType);
+
   public abstract void setId(int id);
-}
+
+  protected static final String TASK = "task";
+
+  static final ThreadLocal<NumberFormat> taskIdFormat =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(6);
+          return fmt;
+        }
+      };
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + getId();
+    result = prime * result + getJobId().hashCode();
+    result = prime * result + getTaskType().hashCode();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    TaskId other = (TaskId) obj;
+    if (getId() != other.getId())
+      return false;
+    if (!getJobId().equals(other.getJobId()))
+      return false;
+    if (getTaskType() != other.getTaskType())
+      return false;
+    return true;
+  }
+      
+  @Override
+  public String toString() {
+    StringBuilder builder = new StringBuilder(TASK);
+    JobId jobId = getJobId();
+    builder.append("_").append(jobId.getAppId().getClusterTimestamp());
+    builder.append("_").append(
+        JobId.jobIdFormat.get().format(jobId.getAppId().getId()));
+    builder.append("_");
+    builder.append(getTaskType() == TaskType.MAP ? "m" : "r").append("_");
+    builder.append(taskIdFormat.get().format(getId()));
+    return builder.toString();
+  }
+
+  @Override
+  public int compareTo(TaskId other) {
+    int jobIdComp = this.getJobId().compareTo(other.getJobId());
+    if (jobIdComp == 0) {
+      if (this.getTaskType() == other.getTaskType()) {
+        return this.getId() - other.getId();
+      } else {
+        return this.getTaskType().compareTo(other.getTaskType());
+      }
+    } else {
+      return jobIdComp;
+    }
+  }
+}

+ 11 - 35
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobIdPBImpl.java

@@ -18,35 +18,21 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records.impl.pb;
 
-import java.text.NumberFormat;
-
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.JobIdProto;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.JobIdProtoOrBuilder;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
     
-public class JobIdPBImpl extends ProtoBase<JobIdProto> implements JobId {
+public class JobIdPBImpl extends JobId {
 
-  protected static final String JOB = "job";
-  protected static final char SEPARATOR = '_';
-  protected static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(4);
-  }
-  
-  
   JobIdProto proto = JobIdProto.getDefaultInstance();
   JobIdProto.Builder builder = null;
   boolean viaProto = false;
   
   private ApplicationId applicationId = null;
-//  boolean hasLocalAppId = false;
-  
-  
+
   public JobIdPBImpl() {
     builder = JobIdProto.newBuilder();
   }
@@ -56,17 +42,17 @@ public class JobIdPBImpl extends ProtoBase<JobIdProto> implements JobId {
     viaProto = true;
   }
 
-  @Override
   public synchronized JobIdProto getProto() {
-  
-      mergeLocalToProto();
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
   private synchronized void mergeLocalToBuilder() {
-    if (this.applicationId != null && !((ApplicationIdPBImpl)this.applicationId).getProto().equals(builder.getAppId()))   {
+    if (this.applicationId != null
+        && !((ApplicationIdPBImpl) this.applicationId).getProto().equals(
+            builder.getAppId())) {
       builder.setAppId(convertToProtoFormat(this.applicationId));
     }
   }
@@ -107,7 +93,6 @@ public class JobIdPBImpl extends ProtoBase<JobIdProto> implements JobId {
       builder.clearAppId();
     }
     this.applicationId = appId;
-//    builder.setAppId(convertToProtoFormat(appId));
   }
   @Override
   public synchronized int getId() {
@@ -121,21 +106,12 @@ public class JobIdPBImpl extends ProtoBase<JobIdProto> implements JobId {
     builder.setId((id));
   }
 
-  private synchronized ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto p) {
+  private ApplicationIdPBImpl convertFromProtoFormat(
+      ApplicationIdProto p) {
     return new ApplicationIdPBImpl(p);
   }
 
-  private synchronized ApplicationIdProto convertToProtoFormat(ApplicationId t) {
-    return ((ApplicationIdPBImpl)t).getProto();
-  }
-  
-  @Override
-  public String toString() {
-    StringBuilder builder = new StringBuilder(JOB);
-    builder.append(SEPARATOR);
-    builder.append(getAppId().getClusterTimestamp());
-    builder.append(SEPARATOR);
-    builder.append(idFormat.format(getId()));
-    return builder.toString();
+  private ApplicationIdProto convertToProtoFormat(ApplicationId t) {
+    return ((ApplicationIdPBImpl) t).getProto();
   }
-}  
+}

+ 12 - 40
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskAttemptIdPBImpl.java

@@ -18,36 +18,19 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records.impl.pb;
 
-
-import java.text.NumberFormat;
-
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskAttemptIdProto;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskAttemptIdProtoOrBuilder;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskIdProto;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
-
-
     
-public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implements TaskAttemptId {
+public class TaskAttemptIdPBImpl extends TaskAttemptId {
   TaskAttemptIdProto proto = TaskAttemptIdProto.getDefaultInstance();
   TaskAttemptIdProto.Builder builder = null;
   boolean viaProto = false;
   
   private TaskId taskId = null;
-  protected static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(6);
-  }
   
-  protected static final NumberFormat jobidFormat = NumberFormat.getInstance();
-  static {
-    jobidFormat.setGroupingUsed(false);
-    jobidFormat.setMinimumIntegerDigits(4);
-  }
   
   
   public TaskAttemptIdPBImpl() {
@@ -59,20 +42,21 @@ public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implement
     viaProto = true;
   }
   
-  public TaskAttemptIdProto getProto() {
+  public synchronized TaskAttemptIdProto getProto() {
       mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
-  private void mergeLocalToBuilder() {
-    if (this.taskId != null && !((TaskIdPBImpl)this.taskId).getProto().equals(builder.getTaskId())) {
+  private synchronized void mergeLocalToBuilder() {
+    if (this.taskId != null
+        && !((TaskIdPBImpl) this.taskId).getProto().equals(builder.getTaskId())) {
       builder.setTaskId(convertToProtoFormat(this.taskId));
     }
   }
 
-  private void mergeLocalToProto() {
+  private synchronized void mergeLocalToProto() {
     if (viaProto) 
       maybeInitBuilder();
     mergeLocalToBuilder();
@@ -80,7 +64,7 @@ public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implement
     viaProto = true;
   }
 
-  private void maybeInitBuilder() {
+  private synchronized void maybeInitBuilder() {
     if (viaProto || builder == null) {
       builder = TaskAttemptIdProto.newBuilder(proto);
     }
@@ -89,18 +73,18 @@ public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implement
     
   
   @Override
-  public int getId() {
+  public synchronized int getId() {
     TaskAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
     return (p.getId());
   }
 
   @Override
-  public void setId(int id) {
+  public synchronized void setId(int id) {
     maybeInitBuilder();
     builder.setId((id));
   }
   @Override
-  public TaskId getTaskId() {
+  public synchronized TaskId getTaskId() {
     TaskAttemptIdProtoOrBuilder p = viaProto ? proto : builder;
     if (this.taskId != null) {
       return this.taskId;
@@ -113,7 +97,7 @@ public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implement
   }
 
   @Override
-  public void setTaskId(TaskId taskId) {
+  public synchronized void setTaskId(TaskId taskId) {
     maybeInitBuilder();
     if (taskId == null)
       builder.clearTaskId();
@@ -127,16 +111,4 @@ public class TaskAttemptIdPBImpl extends ProtoBase<TaskAttemptIdProto> implement
   private TaskIdProto convertToProtoFormat(TaskId t) {
     return ((TaskIdPBImpl)t).getProto();
   }
-  
-  @Override
-  public String toString() {
-    String identifier = (getTaskId() == null) ? "none":
-      getTaskId().getJobId().getAppId().getClusterTimestamp() + "_" +
-      jobidFormat.format(getTaskId().getJobId().getAppId().getId()) + "_" +
-      ((getTaskId().getTaskType() == TaskType.MAP) ? "m" : "r") + "_" +
-      idFormat.format(getTaskId().getId()) + "_" +
-       getId();
-      
-    return "attempt_" + identifier;
-  }
-}  
+}

+ 15 - 42
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskIdPBImpl.java

@@ -18,9 +18,6 @@
 
 package org.apache.hadoop.mapreduce.v2.api.records.impl.pb;
 
-
-import java.text.NumberFormat;
-
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
@@ -29,30 +26,14 @@ import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskIdProto;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskIdProtoOrBuilder;
 import org.apache.hadoop.mapreduce.v2.proto.MRProtos.TaskTypeProto;
 import org.apache.hadoop.mapreduce.v2.util.MRProtoUtils;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
-
 
-    
-public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
+public class TaskIdPBImpl extends TaskId {
   TaskIdProto proto = TaskIdProto.getDefaultInstance();
   TaskIdProto.Builder builder = null;
   boolean viaProto = false;
-  protected static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(6);
-  }
-  
-  protected static final NumberFormat jobidFormat = NumberFormat.getInstance();
-  static {
-    jobidFormat.setGroupingUsed(false);
-    jobidFormat.setMinimumIntegerDigits(4);
-  }
-  
-  
-  private JobId jobId = null;
-  
-  
+
+  private JobId jobId = null;  
+
   public TaskIdPBImpl() {
     builder = TaskIdProto.newBuilder(proto);
   }
@@ -61,7 +42,7 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
     this.proto = proto;
     viaProto = true;
   }
-  
+
   public synchronized TaskIdProto getProto() {
       mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
@@ -70,7 +51,8 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
   }
 
   private synchronized void mergeLocalToBuilder() {
-    if (this.jobId != null && !((JobIdPBImpl)this.jobId).getProto().equals(builder.getJobId()) ) {
+    if (this.jobId != null
+        && !((JobIdPBImpl) this.jobId).getProto().equals(builder.getJobId())) {
       builder.setJobId(convertToProtoFormat(this.jobId));
     }
   }
@@ -89,8 +71,7 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
     }
     viaProto = false;
   }
-    
-  
+
   @Override
   public synchronized int getId() {
     TaskIdProtoOrBuilder p = viaProto ? proto : builder;
@@ -102,6 +83,7 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
     maybeInitBuilder();
     builder.setId((id));
   }
+
   @Override
   public synchronized JobId getJobId() {
     TaskIdProtoOrBuilder p = viaProto ? proto : builder;
@@ -122,6 +104,7 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
       builder.clearJobId();
     this.jobId = jobId;
   }
+
   @Override
   public synchronized TaskType getTaskType() {
     TaskIdProtoOrBuilder p = viaProto ? proto : builder;
@@ -141,29 +124,19 @@ public class TaskIdPBImpl extends ProtoBase<TaskIdProto> implements TaskId {
     builder.setTaskType(convertToProtoFormat(taskType));
   }
 
-  private synchronized JobIdPBImpl convertFromProtoFormat(JobIdProto p) {
+  private JobIdPBImpl convertFromProtoFormat(JobIdProto p) {
     return new JobIdPBImpl(p);
   }
 
-  private synchronized JobIdProto convertToProtoFormat(JobId t) {
+  private JobIdProto convertToProtoFormat(JobId t) {
     return ((JobIdPBImpl)t).getProto();
   }
 
-  private synchronized TaskTypeProto convertToProtoFormat(TaskType e) {
+  private TaskTypeProto convertToProtoFormat(TaskType e) {
     return MRProtoUtils.convertToProtoFormat(e);
   }
 
-  private synchronized TaskType convertFromProtoFormat(TaskTypeProto e) {
+  private TaskType convertFromProtoFormat(TaskTypeProto e) {
     return MRProtoUtils.convertFromProtoFormat(e);
   }
-
-  
-  @Override
-  public synchronized String toString() {
-    String jobIdentifier =  (jobId == null) ? "none":
-      jobId.getAppId().getClusterTimestamp() + "_" + 
-      jobidFormat.format(jobId.getAppId().getId()) + "_" + 
-      ((getTaskType() == TaskType.MAP) ? "m":"r") + "_" + idFormat.format(getId());
-    return "task_" + jobIdentifier;
-  }
-}  
+}

+ 56 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java

@@ -0,0 +1,56 @@
+/**
+ * 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.mapreduce.v2.util;
+
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.factories.RecordFactory;
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+
+public class MRBuilderUtils {
+
+  private static final RecordFactory recordFactory = RecordFactoryProvider
+      .getRecordFactory(null);
+
+  public static JobId newJobId(ApplicationId appId, int id) {
+    JobId jobId = recordFactory.newRecordInstance(JobId.class);
+    jobId.setAppId(appId);
+    jobId.setId(id);
+    return jobId;
+  }
+
+  public static TaskId newTaskId(JobId jobId, int id, TaskType taskType) {
+    TaskId taskId = recordFactory.newRecordInstance(TaskId.class);
+    taskId.setJobId(jobId);
+    taskId.setId(id);
+    taskId.setTaskType(taskType);
+    return taskId;
+  }
+
+  public static TaskAttemptId newTaskAttemptId(TaskId taskId, int attemptId) {
+    TaskAttemptId taskAttemptId =
+        recordFactory.newRecordInstance(TaskAttemptId.class);
+    taskAttemptId.setTaskId(taskId);
+    taskAttemptId.setId(attemptId);
+    return taskAttemptId;
+  }
+}

+ 139 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/api/records/TestIds.java

@@ -0,0 +1,139 @@
+/**
+ * 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.mapreduce.v2.api.records;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.Test;
+
+public class TestIds {
+
+  @Test
+  public void testJobId() {
+    long ts1 = 1315890136000l;
+    long ts2 = 1315890136001l;
+    JobId j1 = createJobId(ts1, 2);
+    JobId j2 = createJobId(ts1, 1);
+    JobId j3 = createJobId(ts2, 1);
+    JobId j4 = createJobId(ts1, 2);
+
+    assertTrue(j1.equals(j4));
+    assertFalse(j1.equals(j2));
+    assertFalse(j1.equals(j3));
+
+    assertTrue(j1.compareTo(j4) == 0);
+    assertTrue(j1.compareTo(j2) > 0);
+    assertTrue(j1.compareTo(j3) < 0);
+
+    assertTrue(j1.hashCode() == j4.hashCode());
+    assertFalse(j1.hashCode() == j2.hashCode());
+    assertFalse(j1.hashCode() == j3.hashCode());
+
+    JobId j5 = createJobId(ts1, 231415);
+    assertEquals("job_" + ts1 + "_0002", j1.toString());
+    assertEquals("job_" + ts1 + "_231415", j5.toString());
+  }
+
+  @Test
+  public void testTaskId() {
+    long ts1 = 1315890136000l;
+    long ts2 = 1315890136001l;
+    TaskId t1 = createTaskId(ts1, 1, 2, TaskType.MAP);
+    TaskId t2 = createTaskId(ts1, 1, 2, TaskType.REDUCE);
+    TaskId t3 = createTaskId(ts1, 1, 1, TaskType.MAP);
+    TaskId t4 = createTaskId(ts1, 1, 2, TaskType.MAP);
+    TaskId t5 = createTaskId(ts2, 1, 1, TaskType.MAP);
+
+    assertTrue(t1.equals(t4));
+    assertFalse(t1.equals(t2));
+    assertFalse(t1.equals(t3));
+    assertFalse(t1.equals(t5));
+
+    assertTrue(t1.compareTo(t4) == 0);
+    assertTrue(t1.compareTo(t2) < 0);
+    assertTrue(t1.compareTo(t3) > 0);
+    assertTrue(t1.compareTo(t5) < 0);
+
+    assertTrue(t1.hashCode() == t4.hashCode());
+    assertFalse(t1.hashCode() == t2.hashCode());
+    assertFalse(t1.hashCode() == t3.hashCode());
+    assertFalse(t1.hashCode() == t5.hashCode());
+
+    TaskId t6 = createTaskId(ts1, 324151, 54643747, TaskType.REDUCE);
+    assertEquals("task_" + ts1 + "_0001_m_000002", t1.toString());
+    assertEquals("task_" + ts1 + "_324151_r_54643747", t6.toString());
+  }
+
+  @Test
+  public void testTaskAttemptId() {
+    long ts1 = 1315890136000l;
+    long ts2 = 1315890136001l;
+    TaskAttemptId t1 = createTaskAttemptId(ts1, 2, 2, TaskType.MAP, 2);
+    TaskAttemptId t2 = createTaskAttemptId(ts1, 2, 2, TaskType.REDUCE, 2);
+    TaskAttemptId t3 = createTaskAttemptId(ts1, 2, 2, TaskType.MAP, 3);
+    TaskAttemptId t4 = createTaskAttemptId(ts1, 2, 2, TaskType.MAP, 1);
+    TaskAttemptId t5 = createTaskAttemptId(ts1, 2, 1, TaskType.MAP, 3);
+    TaskAttemptId t6 = createTaskAttemptId(ts1, 2, 2, TaskType.MAP, 2);
+
+    assertTrue(t1.equals(t6));
+    assertFalse(t1.equals(t2));
+    assertFalse(t1.equals(t3));
+    assertFalse(t1.equals(t5));
+
+    assertTrue(t1.compareTo(t6) == 0);
+    assertTrue(t1.compareTo(t2) < 0);
+    assertTrue(t1.compareTo(t3) < 0);
+    assertTrue(t1.compareTo(t4) > 0);
+    assertTrue(t1.compareTo(t5) > 0);
+
+    assertTrue(t1.hashCode() == t6.hashCode());
+    assertFalse(t1.hashCode() == t2.hashCode());
+    assertFalse(t1.hashCode() == t3.hashCode());
+    assertFalse(t1.hashCode() == t5.hashCode());
+
+    TaskAttemptId t7 =
+        createTaskAttemptId(ts2, 5463346, 4326575, TaskType.REDUCE, 54375);
+    assertEquals("attempt_" + ts1 + "_0002_m_000002_2", t1.toString());
+    assertEquals("attempt_" + ts2 + "_5463346_r_4326575_54375", t7.toString());
+
+  }
+
+  private JobId createJobId(long clusterTimestamp, int idInt) {
+    return MRBuilderUtils.newJobId(
+        BuilderUtils.newApplicationId(clusterTimestamp, idInt), idInt);
+  }
+
+  private TaskId createTaskId(long clusterTimestamp, int jobIdInt,
+      int taskIdInt, TaskType taskType) {
+    return MRBuilderUtils.newTaskId(createJobId(clusterTimestamp, jobIdInt),
+        taskIdInt, taskType);
+  }
+
+  private TaskAttemptId createTaskAttemptId(long clusterTimestamp,
+      int jobIdInt, int taskIdInt, TaskType taskType, int taskAttemptIdInt) {
+    return MRBuilderUtils.newTaskAttemptId(
+        createTaskId(clusterTimestamp, jobIdInt, taskIdInt, taskType),
+        taskAttemptIdInt);
+  }
+}

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

@@ -535,7 +535,7 @@ public class JobClient extends CLI {
     try {
       conf.setBooleanIfUnset("mapred.mapper.new-api", false);
       conf.setBooleanIfUnset("mapred.reducer.new-api", false);
-      Job job = Job.getInstance(cluster, conf);
+      Job job = Job.getInstance(conf);
       job.submit();
       return new NetworkedJob(job);
     } catch (InterruptedException ie) {

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

@@ -340,7 +340,14 @@ public class ReduceTask extends Task {
     // Initialize the codec
     codec = initCodec();
     RawKeyValueIterator rIter = null;
-    boolean isLocal = "local".equals(job.get(MRConfig.MASTER_ADDRESS, "local"));
+    
+    boolean isLocal = false; 
+    // local iff framework == classic && master address == local
+    String framework = job.get(MRConfig.FRAMEWORK_NAME, MRConfig.CLASSIC_FRAMEWORK_NAME);
+    if (framework.equals(MRConfig.CLASSIC_FRAMEWORK_NAME)) {
+    	isLocal = "local".equals(job.get(MRConfig.MASTER_ADDRESS, "local"));        	
+    }
+    
     if (!isLocal) {
       Class combinerClass = conf.getCombinerClass();
       CombineOutputCollector combineCollector = 

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Cluster.java

@@ -112,7 +112,7 @@ public class Cluster {
   private Job[] getJobs(JobStatus[] stats) throws IOException {
     List<Job> jobs = new ArrayList<Job>();
     for (JobStatus stat : stats) {
-      jobs.add(new Job(this, stat, new JobConf(stat.getJobFile())));
+      jobs.add(Job.getInstance(this, stat, new JobConf(stat.getJobFile())));
     }
     return jobs.toArray(new Job[0]);
   }
@@ -152,7 +152,7 @@ public class Cluster {
   public Job getJob(JobID jobId) throws IOException, InterruptedException {
     JobStatus status = client.getJobStatus(jobId);
     if (status != null) {
-      return new Job(this, status, new JobConf(status.getJobFile()));
+      return Job.getInstance(this, status, new JobConf(status.getJobFile()));
     }
     return null;
   }

+ 115 - 26
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java

@@ -31,22 +31,22 @@ import java.net.URLConnection;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
 
-import javax.security.auth.login.LoginException;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.filecache.DistributedCache;
+import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.hadoop.mapreduce.util.ConfigUtil;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -130,7 +130,7 @@ public class Job extends JobContextImpl implements JobContext {
 
   @Deprecated
   public Job(Configuration conf) throws IOException {
-    this(new Cluster(conf), conf);
+    this(new JobConf(conf));
   }
 
   @Deprecated
@@ -139,18 +139,13 @@ public class Job extends JobContextImpl implements JobContext {
     setJobName(jobName);
   }
 
-  Job(Cluster cluster) throws IOException {
-    this(cluster, new Configuration());
-  }
-
-  Job(Cluster cluster, Configuration conf) throws IOException {
+  Job(JobConf conf) throws IOException {
     super(conf, null);
-    this.cluster = cluster;
+    this.cluster = null;
   }
 
-  Job(Cluster cluster, JobStatus status,
-             Configuration conf) throws IOException {
-    this(cluster, conf);
+  Job(JobStatus status, JobConf conf) throws IOException {
+    this(conf);
     setJobID(status.getJobID());
     this.status = status;
     state = JobState.RUNNING;
@@ -170,7 +165,13 @@ public class Job extends JobContextImpl implements JobContext {
   }
       
   /**
-   * Creates a new {@link Job} with no particular {@link Cluster} .
+   * Creates a new {@link Job} with no particular {@link Cluster} and a 
+   * given {@link Configuration}.
+   * 
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
+   * 
    * A Cluster will be created from the conf parameter only when it's needed.
    * 
    * @param conf the configuration
@@ -179,13 +180,18 @@ public class Job extends JobContextImpl implements JobContext {
    */
   public static Job getInstance(Configuration conf) throws IOException {
     // create with a null Cluster
-    return new Job(null, conf);
+    JobConf jobConf = new JobConf(conf);
+    return new Job(jobConf);
   }
 
       
   /**
    * Creates a new {@link Job} with no particular {@link Cluster} and a given jobName.
    * A Cluster will be created from the conf parameter only when it's needed.
+   *
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
    * 
    * @param conf the configuration
    * @return the {@link Job} , with no connection to a cluster yet.
@@ -194,25 +200,92 @@ public class Job extends JobContextImpl implements JobContext {
   public static Job getInstance(Configuration conf, String jobName)
            throws IOException {
     // create with a null Cluster
-    Job result = new Job(null, conf);
+    Job result = getInstance(conf);
     result.setJobName(jobName);
     return result;
   }
   
-  public static Job getInstance(Cluster cluster) throws IOException {
-     return new Job(cluster);
+  /**
+   * Creates a new {@link Job} with no particular {@link Cluster} and given
+   * {@link Configuration} and {@link JobStatus}.
+   * A Cluster will be created from the conf parameter only when it's needed.
+   * 
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
+   * 
+   * @param status job status
+   * @param conf job configuration
+   * @return the {@link Job} , with no connection to a cluster yet.
+   * @throws IOException
+   */
+  public static Job getInstance(JobStatus status, Configuration conf) 
+  throws IOException {
+    return new Job(status, new JobConf(conf));
+  }
+
+  /**
+   * Creates a new {@link Job} with no particular {@link Cluster}.
+   * A Cluster will be created from the conf parameter only when it's needed.
+   *
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
+   * 
+   * @param ignored
+   * @return the {@link Job} , with no connection to a cluster yet.
+   * @throws IOException
+   * @deprecated Use {@link #getInstance()}
+   */
+  @Deprecated
+  public static Job getInstance(Cluster ignored) throws IOException {
+    return getInstance();
   }
   
-  public static Job getInstance(Cluster cluster, Configuration conf) 
+  /**
+   * Creates a new {@link Job} with no particular {@link Cluster} and given
+   * {@link Configuration}.
+   * A Cluster will be created from the conf parameter only when it's needed.
+   * 
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
+   * 
+   * @param ignored
+   * @param conf job configuration
+   * @return the {@link Job} , with no connection to a cluster yet.
+   * @throws IOException
+   * @deprecated Use {@link #getInstance(Configuration)}
+   */
+  @Deprecated
+  public static Job getInstance(Cluster ignored, Configuration conf) 
       throws IOException {
-    return new Job(cluster, conf);
+    return getInstance(conf);
   }
   
+  /**
+   * Creates a new {@link Job} with no particular {@link Cluster} and given
+   * {@link Configuration} and {@link JobStatus}.
+   * A Cluster will be created from the conf parameter only when it's needed.
+   * 
+   * The <code>Job</code> makes a copy of the <code>Configuration</code> so 
+   * that any necessary internal modifications do not reflect on the incoming 
+   * parameter.
+   * 
+   * @param cluster cluster
+   * @param status job status
+   * @param conf job configuration
+   * @return the {@link Job} , with no connection to a cluster yet.
+   * @throws IOException
+   */
+  @Private
   public static Job getInstance(Cluster cluster, JobStatus status, 
       Configuration conf) throws IOException {
-    return new Job(cluster, status, conf);
+    Job job = getInstance(status, conf);
+    job.setCluster(cluster);
+    return job;
   }
-  
+
   private void ensureState(JobState state) throws IllegalStateException {
     if (state != this.state) {
       throw new IllegalStateException("Job in state "+ this.state + 
@@ -254,6 +327,10 @@ public class Job extends JobContextImpl implements JobContext {
     updateStatus();
     return status;
   }
+  
+  private void setStatus(JobStatus status) {
+    this.status = status;
+  }
 
   /**
    * Returns the current state of the Job.
@@ -354,6 +431,12 @@ public class Job extends JobContextImpl implements JobContext {
     return status.isRetired();
   }
 
+  /** Only for mocks in unit tests. */
+  @Private
+  private void setCluster(Cluster cluster) {
+    this.cluster = cluster;
+  }
+
   /**
    * Dump stats to screen.
    */
@@ -1055,6 +1138,12 @@ public class Job extends JobContextImpl implements JobContext {
     return cluster != null;
   }
 
+  /** Only for mocking via unit tests. */
+  @Private
+  public JobSubmitter getJobSubmitter(FileSystem fs, 
+      ClientProtocol submitClient) throws IOException {
+    return new JobSubmitter(fs, submitClient);
+  }
   /**
    * Submit the job to the cluster and return immediately.
    * @throws IOException
@@ -1064,8 +1153,8 @@ public class Job extends JobContextImpl implements JobContext {
     ensureState(JobState.DEFINE);
     setUseNewAPI();
     connect();
-    final JobSubmitter submitter = new JobSubmitter(cluster.getFileSystem(),
-        cluster.getClient());
+    final JobSubmitter submitter = 
+        getJobSubmitter(cluster.getFileSystem(), cluster.getClient());
     status = ugi.doAs(new PrivilegedExceptionAction<JobStatus>() {
       public JobStatus run() throws IOException, InterruptedException, 
       ClassNotFoundException {
@@ -1114,7 +1203,7 @@ public class Job extends JobContextImpl implements JobContext {
       throws IOException, InterruptedException {
     String lastReport = null;
     Job.TaskStatusFilter filter;
-    Configuration clientConf = cluster.getConf();
+    Configuration clientConf = getConfiguration();
     filter = Job.getTaskOutputFilter(clientConf);
     JobID jobId = getJobID();
     LOG.info("Running job: " + jobId);

+ 0 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobSubmitter.java

@@ -319,7 +319,6 @@ class JobSubmitter {
    * @throws InterruptedException
    * @throws IOException
    */
-  @SuppressWarnings("unchecked")
   JobStatus submitJobInternal(Job job, Cluster cluster) 
   throws ClassNotFoundException, InterruptedException, IOException {
 

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

@@ -65,6 +65,9 @@ public interface MRConfig {
     "mapreduce.jobtracker.kerberos.principal";
 
   public static final String FRAMEWORK_NAME  = "mapreduce.framework.name";
+  public static final String CLASSIC_FRAMEWORK_NAME  = "classic";
+  public static final String YARN_FRAMEWORK_NAME  = "yarn";
+
   public static final String TASK_LOCAL_OUTPUT_CLASS =
   "mapreduce.task.local.output.class";
 }

+ 6 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/JobContextImpl.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration.IntegerRanges;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobContext;
@@ -60,7 +61,11 @@ public class JobContextImpl implements JobContext {
   protected final Credentials credentials;
   
   public JobContextImpl(Configuration conf, JobID jobId) {
-    this.conf = new org.apache.hadoop.mapred.JobConf(conf);
+    if (conf instanceof JobConf) {
+      this.conf = (JobConf)conf;
+    } else {
+      this.conf = new JobConf(conf);
+    }
     this.jobId = jobId;
     this.credentials = this.conf.getCredentials();
     try {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/tools/CLI.java

@@ -215,7 +215,7 @@ public class CLI extends Configured implements Tool {
     // Submit the request
     try {
       if (submitJobFile != null) {
-        Job job = Job.getInstance(cluster, new JobConf(submitJobFile));
+        Job job = Job.getInstance(new JobConf(submitJobFile));
         job.submit();
         System.out.println("Created job " + job.getJobID());
         exitCode = 0;

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobMonitorAndPrint.java

@@ -64,7 +64,7 @@ public class TestJobMonitorAndPrint extends TestCase {
     when(cluster.getClient()).thenReturn(clientProtocol);
     JobStatus jobStatus = new JobStatus(new JobID("job_000", 1), 0f, 0f, 0f, 0f, 
         State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
-    job = new Job(cluster, jobStatus, conf);
+    job = Job.getInstance(cluster, jobStatus, conf);
     job = spy(job);
   }
 

+ 2 - 9
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java

@@ -139,18 +139,11 @@ public class HsController extends AppController {
   /**
    * @return the page that will be used to render the /conf page
    */
+  @Override
   protected Class<? extends View> confPage() {
     return HsConfPage.class;
   }
-  
-  /**
-   * Render the /conf page
-   */
-  public void conf() {
-    requireJob();
-    render(confPage());
-  }
-  
+
   /**
    * @return the page about the current server.
    */

+ 86 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java

@@ -92,6 +92,60 @@ public class TestJobHistoryEvents {
         parsedJob.getState());
   }
 
+  /**
+   * Verify that all the events are flushed on stopping the HistoryHandler
+   * @throws Exception
+   */
+  @Test
+  public void testEventsFlushOnStop() throws Exception {
+
+    Configuration conf = new Configuration();
+    conf.set(MRJobConfig.USER_NAME, "test");
+    MRApp app = new MRAppWithSpecialHistoryHandler(1, 0, true, this
+        .getClass().getName(), true);
+    app.submit(conf);
+    Job job = app.getContext().getAllJobs().values().iterator().next();
+    JobId jobId = job.getID();
+    LOG.info("JOBID is " + TypeConverter.fromYarn(jobId).toString());
+    app.waitForState(job, JobState.SUCCEEDED);
+
+    // make sure all events are flushed
+    app.waitForState(Service.STATE.STOPPED);
+    /*
+     * Use HistoryContext to read logged events and verify the number of
+     * completed maps
+     */
+    HistoryContext context = new JobHistory();
+    ((JobHistory) context).init(conf);
+    Job parsedJob = context.getJob(jobId);
+    Assert.assertEquals("CompletedMaps not correct", 1, parsedJob
+        .getCompletedMaps());
+
+    Map<TaskId, Task> tasks = parsedJob.getTasks();
+    Assert.assertEquals("No of tasks not correct", 1, tasks.size());
+    verifyTask(tasks.values().iterator().next());
+
+    Map<TaskId, Task> maps = parsedJob.getTasks(TaskType.MAP);
+    Assert.assertEquals("No of maps not correct", 1, maps.size());
+
+    Assert.assertEquals("Job state not currect", JobState.SUCCEEDED,
+        parsedJob.getState());
+  }
+
+  @Test
+  public void testJobHistoryEventHandlerIsFirstServiceToStop() {
+    MRApp app = new MRAppWithSpecialHistoryHandler(1, 0, true, this
+        .getClass().getName(), true);
+    Configuration conf = new Configuration();
+    app.init(conf);
+    Service[] services = app.getServices().toArray(new Service[0]);
+    // Verifying that it is the last to be added is same as verifying that it is
+    // the first to be stopped. CompositeService related tests already validate
+    // this.
+    Assert.assertEquals("JobHistoryEventHandler",
+        services[services.length - 1].getName());
+  }
+
   private void verifyTask(Task task) {
     Assert.assertEquals("Task state not currect", TaskState.SUCCEEDED,
         task.getState());
@@ -116,14 +170,43 @@ public class TestJobHistoryEvents {
     @Override
     protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
         AppContext context) {
-      JobHistoryEventHandler eventHandler = new JobHistoryEventHandler(context, 
-          getStartCount());
+      JobHistoryEventHandler eventHandler = new JobHistoryEventHandler(
+          context, getStartCount());
       return eventHandler;
     }
   }
-  
+
+  /**
+   * MRapp with special HistoryEventHandler that writes events only during stop.
+   * This is to simulate events that don't get written by the eventHandling
+   * thread due to say a slow DFS and verify that they are flushed during stop.
+   */
+  private static class MRAppWithSpecialHistoryHandler extends MRApp {
+
+    public MRAppWithSpecialHistoryHandler(int maps, int reduces,
+        boolean autoComplete, String testName, boolean cleanOnStart) {
+      super(maps, reduces, autoComplete, testName, cleanOnStart);
+    }
+
+    @Override
+    protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
+        AppContext context) {
+      return new JobHistoryEventHandler(context, getStartCount()) {
+        @Override
+        public void start() {
+          // Don't start any event draining thread.
+          super.eventHandlingThread = new Thread();
+          super.eventHandlingThread.start();
+        }
+      };
+    }
+
+  }
+
   public static void main(String[] args) throws Exception {
     TestJobHistoryEvents t = new TestJobHistoryEvents();
     t.testHistoryEvents();
+    t.testEventsFlushOnStop();
+    t.testJobHistoryEventHandlerIsFirstServiceToStop();
   }
 }

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

@@ -23,6 +23,7 @@ import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -74,8 +75,10 @@ public class ClientCache {
 
   private MRClientProtocol instantiateHistoryProxy()
   throws IOException {
-	final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
-	          JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
+    final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS);
+    if (StringUtils.isEmpty(serviceAddr)) {
+      return null;
+    }
     LOG.info("Connecting to HistoryServer at: " + serviceAddr);
     final Configuration myConf = new Configuration(conf);
     myConf.setClass(YarnConfiguration.YARN_SECURITY_INFO,

+ 28 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -63,7 +63,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
-import org.apache.hadoop.yarn.exceptions.impl.pb.YarnRemoteExceptionPBImpl;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
@@ -86,6 +85,7 @@ class ClientServiceDelegate {
   private boolean forceRefresh;
   private MRClientProtocol realProxy = null;
   private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
+  private static String UNKNOWN_USER = "Unknown User";
 
   ClientServiceDelegate(Configuration conf, ResourceMgrDelegate rm, 
       JobID jobId, MRClientProtocol historyServerProxy) {
@@ -126,7 +126,12 @@ class ClientServiceDelegate {
     // and redirect to the history server.
     ApplicationReport application = rm.getApplicationReport(appId);
     String serviceAddr = null;
-    while (ApplicationState.RUNNING.equals(application.getState())) {
+    while (application == null || ApplicationState.RUNNING.equals(application.getState())) {
+      if (application == null) {
+        LOG.info("Could not get Job info from RM for job " + jobId
+            + ". Redirecting to job history server.");
+        return checkAndGetHSProxy(UNKNOWN_USER, JobState.NEW);
+      }
       try {
         if (application.getHost() == null || "".equals(application.getHost())) {
           LOG.debug("AM not assigned to Job. Waiting to get the AM ...");
@@ -163,6 +168,11 @@ class ClientServiceDelegate {
           throw new YarnException(e1);
         }
         application = rm.getApplicationReport(appId);
+        if (application == null) {
+          LOG.info("Could not get Job info from RM for job " + jobId
+              + ". Redirecting to job history server.");
+          return checkAndGetHSProxy(UNKNOWN_USER, JobState.RUNNING);
+        }
       } catch (InterruptedException e) {
         LOG.warn("getProxy() call interruped", e);
         throw new YarnException(e);
@@ -176,7 +186,7 @@ class ClientServiceDelegate {
     
     String user = application.getUser();
     if (user == null) {
-      throw new YarnRemoteExceptionPBImpl("User is not set in the application report");
+      throw RPCUtil.getRemoteException("User is not set in the application report");
     }
     if (application.getState() == ApplicationState.NEW ||
         application.getState() == ApplicationState.SUBMITTED) {
@@ -199,11 +209,19 @@ class ClientServiceDelegate {
     if (application.getState() == ApplicationState.SUCCEEDED) {
       LOG.info("Application state is completed. " +
           "Redirecting to job history server");
-      realProxy = historyServerProxy;
+      realProxy = checkAndGetHSProxy(user, JobState.SUCCEEDED);
     }
     return realProxy;
   }
 
+  private MRClientProtocol checkAndGetHSProxy(String user, JobState state) {
+    if (null == historyServerProxy) {
+      LOG.warn("Job History Server is not configured.");
+      return getNotRunningJob(user, state);
+    }
+    return historyServerProxy;
+  }
+
   private void instantiateAMProxy(final String serviceAddr) throws IOException {
     UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
     LOG.trace("Connecting to ApplicationMaster at: " + serviceAddr);
@@ -236,11 +254,14 @@ class ClientServiceDelegate {
       try {
         return methodOb.invoke(getProxy(), args);
       } catch (YarnRemoteException yre) {
-        LOG.warn("Exception thrown by remote end.");
-        LOG.warn(RPCUtil.toString(yre));
+        LOG.warn("Exception thrown by remote end.", yre);
         throw yre;
       } catch (InvocationTargetException e) {
-        //TODO Finite # of errors before giving up?
+        if (e.getTargetException() instanceof YarnRemoteException) {
+          LOG.warn("Exception thrown by remote end.", e
+              .getTargetException());
+          throw (YarnRemoteException) e.getTargetException();
+        }
         LOG.info("Failed to contact AM/History for job " + jobId
             + "  Will retry..", e.getTargetException());
         forceRefresh = true;

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

@@ -74,16 +74,16 @@ import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
 public class ResourceMgrDelegate {
   private static final Log LOG = LogFactory.getLog(ResourceMgrDelegate.class);
       
-  private Configuration conf;
+  private YarnConfiguration conf;
   ClientRMProtocol applicationsManager;
   private ApplicationId applicationId;
   private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
 
-  public ResourceMgrDelegate(Configuration conf) {
+  public ResourceMgrDelegate(YarnConfiguration conf) {
     this.conf = conf;
-    YarnRPC rpc = YarnRPC.create(conf);
+    YarnRPC rpc = YarnRPC.create(this.conf);
     InetSocketAddress rmAddress =
-        NetUtils.createSocketAddr(conf.get(
+        NetUtils.createSocketAddr(this.conf.get(
             YarnConfiguration.RM_ADDRESS,
             YarnConfiguration.DEFAULT_RM_ADDRESS));
     LOG.info("Connecting to ResourceManager at " + rmAddress);

+ 6 - 25
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java

@@ -20,16 +20,13 @@ package org.apache.hadoop.mapred;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.Vector;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
@@ -43,7 +40,6 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.JobStatus;
-import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.QueueAclsInfo;
 import org.apache.hadoop.mapreduce.QueueInfo;
@@ -62,7 +58,6 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -91,7 +86,7 @@ public class YARNRunner implements ClientProtocol {
   private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   private ResourceMgrDelegate resMgrDelegate;
   private ClientCache clientCache;
-  private YarnConfiguration conf;
+  private Configuration conf;
   private final FileContext defaultFileContext;
 
   /**
@@ -99,22 +94,21 @@ public class YARNRunner implements ClientProtocol {
    * yarn
    * @param conf the configuration object for the client
    */
-  public YARNRunner(YarnConfiguration conf) {
-   this(conf, new ResourceMgrDelegate(conf));
+  public YARNRunner(Configuration conf) {
+   this(conf, new ResourceMgrDelegate(new YarnConfiguration(conf)));
   }
 
   /**
-   * Similar to {@link #YARNRunner(YarnConfiguration)} but allowing injecting 
+   * Similar to {@link #YARNRunner(Configuration)} but allowing injecting 
    * {@link ResourceMgrDelegate}. Enables mocking and testing.
    * @param conf the configuration object for the client
    * @param resMgrDelegate the resourcemanager client handle.
    */
-  public YARNRunner(YarnConfiguration conf, ResourceMgrDelegate resMgrDelegate) {
+  public YARNRunner(Configuration conf, ResourceMgrDelegate resMgrDelegate) {
     this.conf = conf;
     try {
       this.resMgrDelegate = resMgrDelegate;
-      this.clientCache = new ClientCache(this.conf,
-          resMgrDelegate);
+      this.clientCache = new ClientCache(this.conf, resMgrDelegate);
       this.defaultFileContext = FileContext.getFileContext(this.conf);
     } catch (UnsupportedFileSystemException ufe) {
       throw new RuntimeException("Error in instantiating YarnClient", ufe);
@@ -223,23 +217,10 @@ public class YARNRunner implements ClientProtocol {
       throw new YarnException(e);
     }
 
-    // XXX Remove
-    Path submitJobDir = new Path(jobSubmitDir);
-    FileContext defaultFS = FileContext.getFileContext(conf);
-    Path submitJobFile =
-      defaultFS.makeQualified(JobSubmissionFiles.getJobConfPath(submitJobDir));
-    FSDataInputStream in = defaultFS.open(submitJobFile);
-    conf.addResource(in);
-    // ---
-
     // Construct necessary information to start the MR AM
     ApplicationSubmissionContext appContext = 
       createApplicationSubmissionContext(conf, jobSubmitDir, ts);
     
-    // XXX Remove
-    in.close();
-    // ---
-    
     // Submit to ResourceManager
     ApplicationId applicationId = resMgrDelegate.submitApplication(appContext);
     

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

@@ -25,14 +25,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 public class YarnClientProtocolProvider extends ClientProtocolProvider {
 
   @Override
   public ClientProtocol create(Configuration conf) throws IOException {
-    if ("yarn".equals(conf.get(MRConfig.FRAMEWORK_NAME))) {
-      return new YARNRunner(new YarnConfiguration(conf));
+    if (MRConfig.YARN_FRAMEWORK_NAME.equals(conf.get(MRConfig.FRAMEWORK_NAME))) {
+      return new YARNRunner(conf);
     }
     return null;
   }

+ 3 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java

@@ -119,7 +119,7 @@ public class TestClientRedirect {
   public void testRedirect() throws Exception {
     
     Configuration conf = new YarnConfiguration();
-    conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
     conf.set(YarnConfiguration.RM_ADDRESS, RMADDRESS);
     conf.set(JHAdminConfig.MR_HISTORY_ADDRESS, HSHOSTADDRESS);
     RMService rmService = new RMService("test");
@@ -139,8 +139,8 @@ public class TestClientRedirect {
     Cluster cluster = new Cluster(conf);
     org.apache.hadoop.mapreduce.JobID jobID =
       new org.apache.hadoop.mapred.JobID("201103121733", 1);
-    org.apache.hadoop.mapreduce.Counters counters = cluster.getJob(jobID)
-        .getCounters();
+    org.apache.hadoop.mapreduce.Counters counters = 
+        cluster.getJob(jobID).getCounters();
     validateCounters(counters);
     Assert.assertTrue(amContact);
    

+ 166 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java

@@ -0,0 +1,166 @@
+/**
+ * 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.mapred;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.MRConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportRequest;
+import org.apache.hadoop.mapreduce.v2.api.protocolrecords.GetJobReportResponse;
+import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
+import org.apache.hadoop.mapreduce.v2.api.records.JobState;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
+import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Test;
+
+/**
+ * Tests for ClientServiceDelegate.java
+ */
+
+public class TestClientServiceDelegate {
+  private JobID oldJobId = JobID.forName("job_1315895242400_2");
+  private org.apache.hadoop.mapreduce.v2.api.records.JobId jobId = TypeConverter
+      .toYarn(oldJobId);
+
+  @Test
+  public void testUnknownAppInRM() throws Exception {
+    MRClientProtocol historyServerProxy = mock(MRClientProtocol.class);
+    when(historyServerProxy.getJobReport(getJobReportRequest())).thenReturn(
+        getJobReportResponse());
+    ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate(
+        historyServerProxy, getRMDelegate());
+
+    JobStatus jobStatus = clientServiceDelegate.getJobStatus(oldJobId);
+    Assert.assertNotNull(jobStatus);
+  }
+
+  @Test
+  public void testRemoteExceptionFromHistoryServer() throws Exception {
+
+    MRClientProtocol historyServerProxy = mock(MRClientProtocol.class);
+    when(historyServerProxy.getJobReport(getJobReportRequest())).thenThrow(
+        RPCUtil.getRemoteException("Job ID doesnot Exist"));
+
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    when(rm.getApplicationReport(TypeConverter.toYarn(oldJobId).getAppId()))
+        .thenReturn(null);
+
+    ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate(
+        historyServerProxy, rm);
+
+    try {
+      clientServiceDelegate.getJobStatus(oldJobId);
+      Assert.fail("Invoke should throw exception after retries.");
+    } catch (YarnRemoteException e) {
+      Assert.assertEquals("Job ID doesnot Exist", e.getMessage());
+    }
+  }
+
+  @Test
+  public void testRetriesOnConnectionFailure() throws Exception {
+
+    MRClientProtocol historyServerProxy = mock(MRClientProtocol.class);
+    when(historyServerProxy.getJobReport(getJobReportRequest())).thenThrow(
+        new RuntimeException("1")).thenThrow(new RuntimeException("2"))
+        .thenThrow(new RuntimeException("3"))
+        .thenReturn(getJobReportResponse());
+
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    when(rm.getApplicationReport(TypeConverter.toYarn(oldJobId).getAppId()))
+        .thenReturn(null);
+
+    ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate(
+        historyServerProxy, rm);
+
+    JobStatus jobStatus = clientServiceDelegate.getJobStatus(oldJobId);
+    Assert.assertNotNull(jobStatus);
+  }
+
+  @Test
+  public void testHistoryServerNotConfigured() throws Exception {
+    //RM doesn't have app report and job History Server is not configured
+    ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate(
+        null, getRMDelegate());
+    JobStatus jobStatus = clientServiceDelegate.getJobStatus(oldJobId);
+    Assert.assertEquals("Unknown User", jobStatus.getUsername());
+    Assert.assertEquals(JobStatus.State.PREP, jobStatus.getState());
+
+    //RM has app report and job History Server is not configured
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    ApplicationReport applicationReport = getApplicationReport();
+    when(rm.getApplicationReport(jobId.getAppId())).thenReturn(
+        applicationReport);
+
+    clientServiceDelegate = getClientServiceDelegate(null, rm);
+    jobStatus = clientServiceDelegate.getJobStatus(oldJobId);
+    Assert.assertEquals(applicationReport.getUser(), jobStatus.getUsername());
+    Assert.assertEquals(JobStatus.State.SUCCEEDED, jobStatus.getState());
+  }
+
+  private GetJobReportRequest getJobReportRequest() {
+    GetJobReportRequest request = Records.newRecord(GetJobReportRequest.class);
+    request.setJobId(jobId);
+    return request;
+  }
+
+  private GetJobReportResponse getJobReportResponse() {
+    GetJobReportResponse jobReportResponse = Records
+        .newRecord(GetJobReportResponse.class);
+    JobReport jobReport = Records.newRecord(JobReport.class);
+    jobReport.setJobId(jobId);
+    jobReport.setJobState(JobState.SUCCEEDED);
+    jobReportResponse.setJobReport(jobReport);
+    return jobReportResponse;
+  }
+
+  private ApplicationReport getApplicationReport() {
+    ApplicationReport applicationReport = Records
+        .newRecord(ApplicationReport.class);
+    applicationReport.setState(ApplicationState.SUCCEEDED);
+    applicationReport.setUser("root");
+    return applicationReport;
+  }
+
+  private ResourceMgrDelegate getRMDelegate() throws YarnRemoteException {
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    when(rm.getApplicationReport(jobId.getAppId())).thenReturn(null);
+    return rm;
+  }
+
+  private ClientServiceDelegate getClientServiceDelegate(
+      MRClientProtocol historyServerProxy, ResourceMgrDelegate rm) {
+    Configuration conf = new YarnConfiguration();
+    conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
+    ClientServiceDelegate clientServiceDelegate = new ClientServiceDelegate(
+        conf, rm, oldJobId, historyServerProxy);
+    return clientServiceDelegate;
+  }
+
+}

+ 6 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.hs.JobHistoryServer;
+import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.MiniYARNCluster;
@@ -59,7 +60,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
 
   @Override
   public void init(Configuration conf) {
-    conf.set(MRConfig.FRAMEWORK_NAME, "yarn");
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
     conf.set(MRJobConfig.USER_NAME, System.getProperty("user.name"));
     conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(),
         "apps_staging_dir/${user.name}/").getAbsolutePath());
@@ -82,6 +83,10 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
     // for corresponding uberized tests.
     conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
 
+    // Set config for JH Server
+    conf.set(JHAdminConfig.MR_HISTORY_ADDRESS, 
+        JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
+    
     super.init(conf);
   }
 

+ 8 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java

@@ -52,6 +52,7 @@ import org.apache.hadoop.mapreduce.Counters;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.TaskAttemptID;
@@ -133,11 +134,15 @@ public class TestMRJobs {
       return;
     }
 
+    Configuration sleepConf = new Configuration(mrCluster.getConfig());
+    // set master address to local to test that local mode applied iff framework == classic and master_address == local
+    sleepConf.set(MRConfig.MASTER_ADDRESS, "local");	
+    
     SleepJob sleepJob = new SleepJob();
-    sleepJob.setConf(mrCluster.getConfig());
-
-    int numReduces = mrCluster.getConfig().getInt("TestMRJobs.testSleepJob.reduces", 2); // or mrCluster.getConfig().getInt(MRJobConfig.NUM_REDUCES, 2);
+    sleepJob.setConf(sleepConf);
 
+    int numReduces = sleepConf.getInt("TestMRJobs.testSleepJob.reduces", 2); // or sleepConf.getConfig().getInt(MRJobConfig.NUM_REDUCES, 2);
+   
     // job with 3 maps (10s) and numReduces reduces (5s), 1 "record" each:
     Job job = sleepJob.createJob(3, numReduces, 10000, 1, 5000, 1);
 

+ 31 - 32
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationAttemptId.java

@@ -60,20 +60,16 @@ public abstract class ApplicationAttemptId implements
   @Unstable
   public abstract void setAttemptId(int attemptId);
 
-  
-  
-  protected static final NumberFormat idFormat = NumberFormat.getInstance();
-  static {
-    idFormat.setGroupingUsed(false);
-    idFormat.setMinimumIntegerDigits(4);
-  }
-
-  protected static final NumberFormat counterFormat = NumberFormat
-      .getInstance();
-  static {
-    counterFormat.setGroupingUsed(false);
-    counterFormat.setMinimumIntegerDigits(6);
-  }
+  static final ThreadLocal<NumberFormat> attemptIdFormat =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(6);
+          return fmt;
+        }
+      };
 
   @Override
   public int hashCode() {
@@ -81,22 +77,25 @@ public abstract class ApplicationAttemptId implements
     final int prime = 31;
     int result = 1;
     ApplicationId appId = getApplicationId();
-    result = prime * result + ((appId == null) ? 0 : appId.hashCode());
+    result = prime * result +  appId.hashCode();
     result = prime * result + getAttemptId();
     return result;
   }
 
   @Override
-  public boolean equals(Object other) {
-    if (other == null)
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
       return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      ApplicationAttemptId otherAttemptId = (ApplicationAttemptId) other;
-      if (this.getApplicationId().equals(otherAttemptId.getApplicationId())) {
-        return this.getAttemptId() == otherAttemptId.getAttemptId();
-      }
-    }
-    return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ApplicationAttemptId other = (ApplicationAttemptId) obj;
+    if (!this.getApplicationId().equals(other.getApplicationId()))
+      return false;
+    if (this.getAttemptId() != other.getAttemptId())
+      return false;
+    return true;
   }
 
   @Override
@@ -109,14 +108,14 @@ public abstract class ApplicationAttemptId implements
       return compareAppIds;
     }
   }
-  
+
   @Override
   public String toString() {
-    String id =
-        (this.getApplicationId() != null) ? this.getApplicationId()
-            .getClusterTimestamp()
-            + "_"
-            + idFormat.format(this.getApplicationId().getId()) : "none";
-    return "appattempt_" + id + "_" + counterFormat.format(getAttemptId());
+    StringBuilder sb = new StringBuilder("appattempt_");
+    sb.append(this.getApplicationId().getClusterTimestamp()).append("_");
+    sb.append(ApplicationId.appIdFormat.get().format(
+        this.getApplicationId().getId()));
+    sb.append("_").append(attemptIdFormat.get().format(getAttemptId()));
+    return sb.toString();
   }
-}
+}

+ 31 - 13
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationId.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api.records;
 
+import java.text.NumberFormat;
+
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
@@ -60,8 +62,20 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
   @Private
   @Unstable
   public abstract void setClusterTimestamp(long clusterTimestamp);
+
   
   
+  static final ThreadLocal<NumberFormat> appIdFormat =
+    new ThreadLocal<NumberFormat>() {
+      @Override
+      public NumberFormat initialValue() {
+        NumberFormat fmt = NumberFormat.getInstance();
+        fmt.setGroupingUsed(false);
+        fmt.setMinimumIntegerDigits(4);
+        return fmt;
+      }
+    };
+
   @Override
   public int compareTo(ApplicationId other) {
     if (this.getClusterTimestamp() - other.getClusterTimestamp() == 0) {
@@ -74,9 +88,10 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
 
   @Override
   public String toString() {
-    return "application_" + this.getClusterTimestamp() + "_" + this.getId();
+    return "application_" + this.getClusterTimestamp() + "_"
+        + appIdFormat.get().format(getId());
   }
-  
+
   @Override
   public int hashCode() {
     // Generated by eclipse.
@@ -90,15 +105,18 @@ public abstract class ApplicationId implements Comparable<ApplicationId> {
   }
 
   @Override
-  public boolean equals(Object other) {
-    if (other == null) return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      ApplicationId otherAppId = (ApplicationId)other;
-      if (this.getClusterTimestamp() == otherAppId.getClusterTimestamp() &&
-          this.getId() == otherAppId.getId()) {
-        return true;
-      }
-    }
-    return false;
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ApplicationId other = (ApplicationId) obj;
+    if (this.getClusterTimestamp() != other.getClusterTimestamp())
+      return false;
+    if (this.getId() != other.getId())
+      return false;
+    return true;
   }
-}
+}

+ 21 - 31
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java

@@ -59,19 +59,6 @@ public abstract class ContainerId implements Comparable<ContainerId>{
   public abstract void setId(int id);
  
   
-  // TODO: Why thread local?
-  // ^ NumberFormat instances are not threadsafe
-  private static final ThreadLocal<NumberFormat> appIdFormat =
-      new ThreadLocal<NumberFormat>() {
-        @Override
-        public NumberFormat initialValue() {
-          NumberFormat fmt = NumberFormat.getInstance();
-          fmt.setGroupingUsed(false);
-          fmt.setMinimumIntegerDigits(4);
-          return fmt;
-        }
-      };
-
   // TODO: fail the app submission if attempts are more than 10 or something
   private static final ThreadLocal<NumberFormat> appAttemptIdFormat =
       new ThreadLocal<NumberFormat>() {
@@ -102,24 +89,24 @@ public abstract class ContainerId implements Comparable<ContainerId>{
     final int prime = 31;
     int result = 1;
     result = prime * result + getId();
-    result = prime * result
-        + ((getApplicationAttemptId() == null) ? 0 : getApplicationAttemptId().hashCode());
+    result = prime * result + getApplicationAttemptId().hashCode();
     return result;
   }
 
   @Override
-  public boolean equals(Object other) {
-    if (other == null) {
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
       return false;
-    }
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      ContainerId otherCId = (ContainerId)other;
-      if (this.getApplicationAttemptId().equals(
-          otherCId.getApplicationAttemptId())) {
-        return this.getId() == otherCId.getId();
-      }
-    }
-    return false;
+    if (getClass() != obj.getClass())
+      return false;
+    ContainerId other = (ContainerId) obj;
+    if (!this.getApplicationAttemptId().equals(other.getApplicationAttemptId()))
+      return false;
+    if (this.getId() != other.getId())
+      return false;
+    return true;
   }
 
   @Override
@@ -133,15 +120,18 @@ public abstract class ContainerId implements Comparable<ContainerId>{
     }
     
   }
-  
+
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder();
+    sb.append("container_");
     ApplicationId appId = getApplicationAttemptId().getApplicationId();
-    sb.append("container_").append(appId.getClusterTimestamp()).append("_");
-    sb.append(appIdFormat.get().format(appId.getId())).append("_");
-    sb.append(appAttemptIdFormat.get().format(getApplicationAttemptId().
-        getAttemptId())).append("_");
+    sb.append(appId.getClusterTimestamp()).append("_");
+    sb.append(ApplicationId.appIdFormat.get().format(appId.getId()))
+        .append("_");
+    sb.append(
+        appAttemptIdFormat.get().format(
+            getApplicationAttemptId().getAttemptId())).append("_");
     sb.append(containerIdFormat.get().format(getId()));
     return sb.toString();
   }

+ 50 - 5
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeId.java

@@ -32,7 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable;
  */
 @Public
 @Stable
-public interface NodeId extends Comparable<NodeId> {
+public abstract class NodeId implements Comparable<NodeId> {
 
   /**
    * Get the <em>hostname</em> of the node.
@@ -40,11 +40,11 @@ public interface NodeId extends Comparable<NodeId> {
    */ 
   @Public
   @Stable
-  String getHost();
+  public abstract String getHost();
   
   @Private
   @Unstable
-  void setHost(String host);
+  public abstract void setHost(String host);
 
   /**
    * Get the <em>port</em> for communicating with the node.
@@ -52,9 +52,54 @@ public interface NodeId extends Comparable<NodeId> {
    */
   @Public
   @Stable
-  int getPort();
+  public abstract int getPort();
   
   @Private
   @Unstable
-  void setPort(int port);
+  public abstract void setPort(int port);
+
+  @Override
+  public String toString() {
+    return this.getHost() + ":" + this.getPort();
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + this.getHost().hashCode();
+    result = prime * result + this.getPort();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj)
+      return true;
+    if (obj == null)
+      return false;
+    if (getClass() != obj.getClass())
+      return false;
+    NodeId other = (NodeId) obj;
+    if (!this.getHost().equals(other.getHost()))
+      return false;
+    if (this.getPort() != other.getPort())
+      return false;
+    return true;
+  }
+
+  @Override
+  public int compareTo(NodeId other) {
+    int hostCompare = this.getHost().compareTo(other.getHost());
+    if (hostCompare == 0) {
+      if (this.getPort() > other.getPort()) {
+        return 1;
+      } else if (this.getPort() < other.getPort()) {
+        return -1;
+      }
+      return 0;
+    }
+    return hostCompare;
+  }
+
 }

+ 5 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationAttemptIdPBImpl.java

@@ -25,7 +25,8 @@ import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProtoOrBuilde
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 
 public class ApplicationAttemptIdPBImpl extends ApplicationAttemptId {
-  ApplicationAttemptIdProto proto = ApplicationAttemptIdProto.getDefaultInstance();
+  ApplicationAttemptIdProto proto = ApplicationAttemptIdProto
+      .getDefaultInstance();
   ApplicationAttemptIdProto.Builder builder = null;
   boolean viaProto = false;
   
@@ -48,7 +49,9 @@ public class ApplicationAttemptIdPBImpl extends ApplicationAttemptId {
   }
 
   private synchronized void mergeLocalToBuilder() {
-    if (this.applicationId != null && !((ApplicationIdPBImpl)applicationId).getProto().equals(builder.getApplicationId())) {
+    if (this.applicationId != null
+        && !((ApplicationIdPBImpl) applicationId).getProto().equals(
+            builder.getApplicationId())) {
       builder.setApplicationId(convertToProtoFormat(this.applicationId));
     }
   }

+ 7 - 59
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeIdPBImpl.java

@@ -20,13 +20,12 @@ package org.apache.hadoop.yarn.api.records.impl.pb;
 
 
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProtoOrBuilder;
 
 
     
-public class NodeIdPBImpl extends ProtoBase<NodeIdProto> implements NodeId {
+public class NodeIdPBImpl extends NodeId {
   NodeIdProto proto = NodeIdProto.getDefaultInstance();
   NodeIdProto.Builder builder = null;
   boolean viaProto = false;
@@ -40,13 +39,13 @@ public class NodeIdPBImpl extends ProtoBase<NodeIdProto> implements NodeId {
     viaProto = true;
   }
   
-  public NodeIdProto getProto() {
+  public synchronized NodeIdProto getProto() {
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
-  private void maybeInitBuilder() {
+  private synchronized void maybeInitBuilder() {
     if (viaProto || builder == null) {
       builder = NodeIdProto.newBuilder(proto);
     }
@@ -54,77 +53,26 @@ public class NodeIdPBImpl extends ProtoBase<NodeIdProto> implements NodeId {
   }
     
   @Override
-  public String getHost() {
+  public synchronized String getHost() {
     NodeIdProtoOrBuilder p = viaProto ? proto : builder;
     return (p.getHost());
   }
 
   @Override
-  public void setHost(String host) {
+  public synchronized void setHost(String host) {
     maybeInitBuilder();
     builder.setHost((host));
   }
 
   @Override
-  public int getPort() {
+  public synchronized int getPort() {
     NodeIdProtoOrBuilder p = viaProto ? proto : builder;
     return (p.getPort());
   }
 
   @Override
-  public void setPort(int port) {
+  public synchronized void setPort(int port) {
     maybeInitBuilder();
     builder.setPort((port));
   }
-
-  @Override
-  public String toString() {
-    return this.getHost() + ":" + this.getPort();
-  }
-  
-  @Override
-  public int hashCode() {
-    final int prime = 31;
-    int result = super.hashCode();
-    String host = this.getHost();
-    result = prime * result + ((host == null) ? 0 : host.hashCode());
-    result = prime * result + this.getPort();
-    return result;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (this == obj)
-      return true;
-    if (!super.equals(obj))
-      return false;
-    if (getClass() != obj.getClass())
-      return false;
-    NodeIdPBImpl other = (NodeIdPBImpl) obj;
-    String host = this.getHost();
-    String otherHost = other.getHost();
-    if (host == null) {
-      if (otherHost != null)
-        return false;
-    } else if (!host.equals(otherHost))
-      return false;
-    if (this.getPort() != other.getPort())
-      return false;
-    return true;
-  }
-
-  @Override
-  public int compareTo(NodeId other) {
-    int hostCompare = this.getHost().compareTo(other.getHost());
-    if (hostCompare == 0) {
-      if (this.getPort() > other.getPort()) {
-        return 1;
-      } else if (this.getPort() < other.getPort()) {
-        return -1;
-      }
-      return 0;
-    }
-    return hostCompare;
-  }
-
 }  

+ 21 - 14
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/Router.java

@@ -25,6 +25,8 @@ import com.google.common.collect.Maps;
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.lang.NoSuchMethodException;
+import java.lang.SecurityException;
 import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
@@ -88,23 +90,28 @@ class Router {
   private Dest addController(WebApp.HTTP httpMethod, String path,
                              Class<? extends Controller> cls,
                              String action, List<String> names) {
-    for (Method method : cls.getDeclaredMethods()) {
-      if (method.getName().equals(action) &&
-          method.getParameterTypes().length == 0 &&
-          Modifier.isPublic(method.getModifiers())) {
-        // TODO: deal with parameters using the names
-        Dest dest = routes.get(path);
-        if (dest == null) {
-          method.setAccessible(true); // avoid any runtime checks
-          dest = new Dest(path, method, cls, names, httpMethod);
-          routes.put(path, dest);
-          return dest;
-        }
-        dest.methods.add(httpMethod);
+    try {
+      // Look for the method in all public methods declared in the class
+      // or inherited by the class.
+      // Note: this does not distinguish methods with the same signature
+      // but different return types.
+      // TODO: We may want to deal with methods that take parameters in the future
+      Method method = cls.getMethod(action, null);
+      Dest dest = routes.get(path);
+      if (dest == null) {
+        method.setAccessible(true); // avoid any runtime checks
+        dest = new Dest(path, method, cls, names, httpMethod);
+        routes.put(path, dest);
         return dest;
       }
+      dest.methods.add(httpMethod);
+      return dest;
+    } catch (NoSuchMethodException nsme) {
+      throw new WebAppException(action + "() not found in " + cls);
+    } catch (SecurityException se) {
+      throw new WebAppException("Security exception thrown for " + action +
+        "() in " + cls);
     }
-    throw new WebAppException(action + "() not found in " + cls);
   }
 
   private void addDefaultView(Dest dest) {

+ 5 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HeaderBlock.java

@@ -23,10 +23,14 @@ import static org.apache.hadoop.yarn.webapp.Params.*;
 public class HeaderBlock extends HtmlBlock {
 
   @Override protected void render(Block html) {
+    String loggedIn = ""; 
+    if (request().getRemoteUser() != null) {
+      loggedIn = "Logged in as: " + request().getRemoteUser();
+    }
     html.
       div("#header.ui-widget").
         div("#user").
-          _("Logged in as: "+ request().getRemoteUser())._().
+          _(loggedIn)._().
         div("#logo").
           img("/static/hadoop-st.png")._().
         h1($(TITLE))._();

+ 29 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationAttemptId.java

@@ -1,3 +1,22 @@
+/**
+ * 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.api;
 
 import junit.framework.Assert;
@@ -32,6 +51,10 @@ public class TestApplicationAttemptId {
     Assert.assertFalse(a1.hashCode() == a3.hashCode());
     Assert.assertFalse(a1.hashCode() == a4.hashCode());
     
+    long ts = System.currentTimeMillis();
+    ApplicationAttemptId a6 = createAppAttemptId(ts, 543627, 33492611);
+    Assert.assertEquals("appattempt_10_0001_000001", a1.toString());
+    Assert.assertEquals("appattempt_" + ts + "_543627_33492611", a6.toString());
   }
 
   private ApplicationAttemptId createAppAttemptId(long clusterTimeStamp,
@@ -45,4 +68,9 @@ public class TestApplicationAttemptId {
     appAttemptId.setAttemptId(attemptId);
     return appAttemptId;
   }
-}
+  
+  public static void main(String[] args) throws Exception {
+    TestApplicationAttemptId t = new TestApplicationAttemptId();
+    t.testApplicationAttemptId();
+  }
+}

+ 23 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestApplicationId.java

@@ -1,3 +1,21 @@
+/**
+ * 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.api;
 
 import junit.framework.Assert;
@@ -26,6 +44,11 @@ public class TestApplicationId {
     Assert.assertTrue(a1.hashCode() == a3.hashCode());
     Assert.assertFalse(a1.hashCode() == a2.hashCode());
     Assert.assertFalse(a2.hashCode() == a4.hashCode());
+    
+    long ts = System.currentTimeMillis();
+    ApplicationId a5 = createAppId(ts, 45436343);
+    Assert.assertEquals("application_10_0001", a1.toString());
+    Assert.assertEquals("application_" + ts + "_45436343", a5.toString());
   }
 
   private ApplicationId createAppId(long clusterTimeStamp, int id) {

+ 25 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java

@@ -1,3 +1,22 @@
+/**
+ * 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.api;
 
 import junit.framework.Assert;
@@ -32,6 +51,12 @@ public class TestContainerId {
     Assert.assertFalse(c1.hashCode() == c2.hashCode());
     Assert.assertFalse(c1.hashCode() == c4.hashCode());
     Assert.assertFalse(c1.hashCode() == c5.hashCode());
+    
+    long ts = System.currentTimeMillis();
+    ContainerId c6 = createContainerId(ts, 36473, 4365472, 25645811);
+    Assert.assertEquals("container_10_0001_01_000001", c1.toString());
+    Assert.assertEquals("container_" + ts + "_36473_4365472_25645811",
+        c6.toString());
   }
 
   private ContainerId createContainerId(long clusterTimestamp, int appIdInt,

+ 56 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestNodeId.java

@@ -0,0 +1,56 @@
+/**
+ * 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.api;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.Records;
+import org.junit.Test;
+
+public class TestNodeId {
+  @Test
+  public void testNodeId() {
+    NodeId nodeId1 = createNodeId("10.18.52.124", 45454);
+    NodeId nodeId2 = createNodeId("10.18.52.125", 45452);
+    NodeId nodeId3 = createNodeId("10.18.52.124", 45454);
+    NodeId nodeId4 = createNodeId("10.18.52.124", 45453);
+
+    Assert.assertTrue(nodeId1.equals(nodeId3));
+    Assert.assertFalse(nodeId1.equals(nodeId2));
+    Assert.assertFalse(nodeId3.equals(nodeId4));
+
+    Assert.assertTrue(nodeId1.compareTo(nodeId3) == 0);
+    Assert.assertTrue(nodeId1.compareTo(nodeId2) < 0);
+    Assert.assertTrue(nodeId3.compareTo(nodeId4) > 0);
+
+    Assert.assertTrue(nodeId1.hashCode() == nodeId3.hashCode());
+    Assert.assertFalse(nodeId1.hashCode() == nodeId2.hashCode());
+    Assert.assertFalse(nodeId3.hashCode() == nodeId4.hashCode());
+
+    Assert.assertEquals("10.18.52.124:45454", nodeId1.toString());
+  }
+
+  private NodeId createNodeId(String host, int port) {
+    NodeId nodeId = Records.newRecord(NodeId.class);
+    nodeId.setHost(host);
+    nodeId.setPort(port);
+    return nodeId;
+  }
+}

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

@@ -97,7 +97,7 @@ public class ResourceTrackerService extends AbstractService implements
   public synchronized void init(Configuration conf) {
     String resourceTrackerBindAddress =
       conf.get(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
-          YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS);
+          YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS);
     resourceTrackerAddress = NetUtils.createSocketAddr(resourceTrackerBindAddress);
 
     RackResolver.init(conf);

+ 80 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java

@@ -18,40 +18,120 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.rmapp;
 
+import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 
+/**
+ * The read interface to an Application in the ResourceManager. Take a 
+ * look at {@link RMAppImpl} for its implementation. This interface 
+ * exposes methods to access various updates in application status/report.
+ */
 public interface RMApp extends EventHandler<RMAppEvent>{
 
+  /**
+   * The application id for this {@link RMApp}.
+   * @return the {@link ApplicationId} for this {@link RMApp}.
+   */
   ApplicationId getApplicationId();
 
+  /**
+   * The current state of the {@link RMApp}.
+   * @return the current state {@link RMAppState} for this application.
+   */
   RMAppState getState();
 
+  /**
+   * The user who submitted this application.
+   * @return the user who submitted the application.
+   */
   String getUser();
 
+  /**
+   * Progress of application.
+   * @return the progress of the {@link RMApp}.
+   */
   float getProgress();
 
+  /**
+   * {@link RMApp} can have multiple application attempts {@link RMAppAttempt}.
+   * This method returns the {@link RMAppAttempt} corresponding to
+   *  {@link ApplicationAttemptId}.
+   * @param appAttemptId the application attempt id
+   * @return  the {@link RMAppAttempt} corresponding to the {@link ApplicationAttemptId}.
+   */
   RMAppAttempt getRMAppAttempt(ApplicationAttemptId appAttemptId);
 
+  /**
+   * Each Application is submitted to a queue decided by {@link 
+   * ApplicationSubmissionContext#setQueue(String)}.
+   * This method returns the queue to which an application was submitted.
+   * @return the queue to which the application was submitted to.
+   */
   String getQueue();
 
+  /**
+   * The name of the application as set in {@link 
+   * ApplicationSubmissionContext#setApplicationName(String)}.
+   * @return the name of the application.
+   */
   String getName();
 
+  /**
+   * {@link RMApp} can have multiple application attempts {@link RMAppAttempt}.
+   * This method returns the current {@link RMAppAttempt}.
+   * @return the current {@link RMAppAttempt} 
+   */
   RMAppAttempt getCurrentAppAttempt();
 
+  /**
+   * To get the status of an application in the RM, this method can be used.
+   * @return the {@link ApplicationReport} detailing the status of the application.
+   */
   ApplicationReport createAndGetApplicationReport();
 
+  /**
+   * Application level metadata is stored in {@link ApplicationStore} whicn 
+   * can persist the information.
+   * @return the {@link ApplicationStore}  for this {@link RMApp}.
+   */
   ApplicationStore getApplicationStore();
 
+  /**
+   * The finish time of the {@link RMApp}
+   * @return the finish time of the application.,
+   */
   long getFinishTime();
 
+  /**
+   * the start time of the application.
+   * @return the start time of the application.
+   */
   long getStartTime();
 
+  /**
+   * The tracking url for the application master.
+   * @return the tracking url for the application master.
+   */
   String getTrackingUrl();
 
+  /**
+   * the diagnostics information for the application master.
+   * @return the diagnostics information for the application master.
+   */
   StringBuilder getDiagnostics();
+  
+  /**
+   * The final state of the AM when unregistering as in 
+   * {@link FinishApplicationMasterRequest#setFinalState(String)}.
+   * @return the final state of the AM as set in 
+   * {@link FinishApplicationMasterRequest#setFinalState(String)}.
+   */
+  String getAMFinalState();
 }

+ 15 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -40,11 +40,10 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
+import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.ApplicationsStore.ApplicationStore;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
@@ -194,6 +193,19 @@ public class RMAppImpl implements RMApp {
     return this.applicationId;
   }
 
+  @Override
+  public String getAMFinalState() {
+    this.readLock.lock();
+    try {
+      if (currentAttempt != null) {
+        return currentAttempt.getAMFinalState();
+      }
+      return "UNKNOWN";
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+  
   @Override
   public RMAppState getState() {
     this.readLock.lock();

+ 71 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java

@@ -26,33 +26,103 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 
+/**
+ * Interface to an Application Attempt in the Resource Manager.
+ * A {@link RMApp} can have multiple app attempts based on 
+ * {@link YarnConfiguration#RM_AM_MAX_RETRIES}. For specific 
+ * implementation take a look at {@link RMAppAttemptImpl}.
+ */
 public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent>{
 
+  /**
+   * Get the application attempt id for this {@link RMAppAttempt}.
+   * @return the {@link ApplicationAttemptId} for this RM attempt.
+   */
   ApplicationAttemptId getAppAttemptId();
 
+  /**
+   * The state of the {@link RMAppAttempt}.
+   * @return the state {@link RMAppAttemptState} of this {@link RMAppAttempt}
+   */
   RMAppAttemptState getAppAttemptState();
-
+  
+  /**
+   * The host on which the {@link RMAppAttempt} is running/ran on.
+   * @return the host on which the {@link RMAppAttempt} ran/is running on.
+   */
   String getHost();
 
+  /**
+   * The rpc port of the {@link RMAppAttempt}.
+   * @return the rpc port of the {@link RMAppAttempt} to which the clients can connect
+   * to.
+   */
   int getRpcPort();
 
+  /**
+   * The url at which the status of the application attempt can be accessed.
+   * @return the url at which the status of the attempt can be accessed.
+   */
   String getTrackingUrl();
 
+  /**
+   * The token required by the clients to talk to the application attempt
+   * @return the token required by the clients to talk to the application attempt
+   */
   String getClientToken();
 
+  /**
+   * Diagnostics information for the application attempt.
+   * @return diagnostics information for the application attempt.
+   */
   StringBuilder getDiagnostics();
 
+  /**
+   * Progress for the application attempt.
+   * @return the progress for this {@link RMAppAttempt}
+   */
   float getProgress();
 
+  /**
+   * The final state set by the AM.
+   * @return the final state that is set by the AM when unregistering itself.
+   */
+  String getAMFinalState();
+  
+  /**
+   * Nodes on which the containers for this {@link RMAppAttempt} ran.
+   * @return the set of nodes that ran any containers from this {@link RMAppAttempt}
+   */
   Set<NodeId> getRanNodes();
 
+  /**
+   * Return a list of the last set of finished containers, resetting the 
+   * finished containers to empty.
+   * @return the list of just finished containers, re setting the finished containers.
+   */
   List<ContainerStatus> pullJustFinishedContainers();
 
+  /**
+   * Return the list of last set of finished containers. This does not reset the 
+   * finished containers.
+   * @return the list of just finished contianers, this does not reset the 
+   * finished containers.
+   */
   List<ContainerStatus> getJustFinishedContainers();
 
+  /**
+   * The container on which the Application Master is running.
+   * @return the {@link Container} on which the application master is running.
+   */
   Container getMasterContainer();
 
+  /**
+   * The application submission context for this {@link RMAppAttempt}.
+   * @return the application submission context for this Application.
+   */
   ApplicationSubmissionContext getSubmissionContext();
 }

+ 12 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -263,6 +263,16 @@ public class RMAppAttemptImpl implements RMAppAttempt {
   public ApplicationSubmissionContext getSubmissionContext() {
     return this.submissionContext;
   }
+  
+  @Override
+  public String getAMFinalState() {
+    this.readLock.lock();
+    try {
+      return this.finalState;
+    } finally {
+      this.readLock.unlock();
+    }
+  }
 
   @Override
   public RMAppAttemptState getAppAttemptState() {
@@ -413,7 +423,8 @@ public class RMAppAttemptImpl implements RMAppAttempt {
   }
 
   private static final class AttemptStartedTransition extends BaseTransition {
-    @Override
+    @SuppressWarnings("unchecked")
+	@Override
     public void transition(RMAppAttemptImpl appAttempt,
         RMAppAttemptEvent event) {
 

+ 3 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java

@@ -23,6 +23,7 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
 
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
@@ -69,7 +70,8 @@ class AppsBlock extends HtmlBlock {
           td(app.getUser().toString()).
           td(app.getName().toString()).
           td(app.getQueue().toString()).
-          td(app.getState().toString()).
+          td(app.getState() == RMAppState.FINISHED ? app.getAMFinalState() : 
+            app.getState().toString()).
           td().
             br().$title(percent)._(). // for sorting
             div(_PROGRESSBAR).

+ 4 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.util.Apps;
@@ -88,7 +89,9 @@ public class RmController extends Controller {
     ResponseInfo info = info("Application Overview").
       _("User:", app.getUser()).
       _("Name:", app.getName()).
-      _("State:", app.getState()).
+      _("State:", (app.getState() == RMAppState.FINISHED ?
+        app.getAMFinalState() : app.getState().toString())
+      ).
       _("Started:", Times.format(app.getStartTime())).
       _("Elapsed:", StringUtils.formatTime(
         Times.elapsed(app.getStartTime(), app.getFinishTime()))).

+ 5 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java

@@ -209,6 +209,11 @@ public abstract class MockAsm extends MockApps {
     public void handle(RMAppEvent event) {
       throw new UnsupportedOperationException("Not supported yet.");      
     }
+
+    @Override
+    public String getAMFinalState() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
   }
 
   public static RMApp newApplication(int i) {

+ 5 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java

@@ -163,6 +163,11 @@ public class MockRMApp implements RMApp {
   }
 
   public void handle(RMAppEvent event) {
+  }
+
+  @Override
+  public String getAMFinalState() {
+    return "UNKNOWN";
   };
 
 }

+ 30 - 1
hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/FairScheduler.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.metrics.MetricsContext;
 import org.apache.hadoop.metrics.MetricsUtil;
 import org.apache.hadoop.metrics.Updater;
 import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * A {@link TaskScheduler} that implements fair sharing.
@@ -97,7 +98,15 @@ public class FairScheduler extends TaskScheduler {
   protected long lastDumpTime;       // Time when we last dumped state to log
   protected long lastHeartbeatTime;  // Time we last ran assignTasks 
   private long lastPreemptCheckTime; // Time we last ran preemptTasksIfNecessary
-  
+
+  /**
+   * A configuration property that controls the ability of submitting jobs to
+   * pools not declared in the scheduler allocation file.
+   */
+  public final static String ALLOW_UNDECLARED_POOLS_KEY =
+    "mapred.fairscheduler.allow.undeclared.pools";
+  private boolean allowUndeclaredPools = false;
+
   /**
    * A class for holding per-job scheduler variables. These always contain the
    * values of the variables at the last update(), and are used along with a
@@ -195,6 +204,8 @@ public class FairScheduler extends TaskScheduler {
           "mapred.fairscheduler.locality.delay.node", defaultDelay);
       rackLocalityDelay = conf.getLong(
           "mapred.fairscheduler.locality.delay.rack", defaultDelay);
+      allowUndeclaredPools = conf.getBoolean(ALLOW_UNDECLARED_POOLS_KEY, true);
+
       if (defaultDelay == -1 && 
           (nodeLocalityDelay == -1 || rackLocalityDelay == -1)) {
         autoComputeLocalityDelay = true; // Compute from heartbeat interval
@@ -1098,4 +1109,22 @@ public class FairScheduler extends TaskScheduler {
   long getLastPreemptionUpdateTime() {
     return lastPreemptionUpdateTime;
   }
+
+  /**
+   * Examines the job's pool name to determine if it is a declared pool name (in
+   * the scheduler allocation file).
+   */
+  @Override
+  public void checkJobSubmission(JobInProgress job)
+      throws UndeclaredPoolException {
+    Set<String> declaredPools = poolMgr.getDeclaredPools();
+    if (!this.allowUndeclaredPools
+        && !declaredPools.contains(poolMgr.getPoolName(job)))
+      throw new UndeclaredPoolException("Pool name: '"
+          + poolMgr.getPoolName(job)
+          + "' is invalid. Add pool name to the fair scheduler allocation "
+          + "file. Valid pools are: "
+          + StringUtils.join(", ", declaredPools));
+  }
+
 }

+ 11 - 0
hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/PoolManager.java

@@ -28,6 +28,8 @@ import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
@@ -114,6 +116,8 @@ public class PoolManager {
   private long lastSuccessfulReload; // Last time we successfully reloaded pools
   private boolean lastReloadAttemptFailed = false;
 
+  private Set<String> declaredPools = new TreeSet<String>();
+
   public PoolManager(FairScheduler scheduler) {
     this.scheduler = scheduler;
   }
@@ -370,6 +374,8 @@ public class PoolManager {
       this.fairSharePreemptionTimeout = fairSharePreemptionTimeout;
       this.defaultMinSharePreemptionTimeout = defaultMinSharePreemptionTimeout;
       this.defaultSchedulingMode = defaultSchedulingMode;
+      this.declaredPools = Collections.unmodifiableSet(new TreeSet<String>(
+          poolNamesInAllocFile));
       for (String name: poolNamesInAllocFile) {
         Pool pool = getPool(name);
         if (poolModes.containsKey(name)) {
@@ -543,4 +549,9 @@ public class PoolManager {
       pool.updateMetrics();
     }
   }
+
+  public synchronized Set<String> getDeclaredPools() {
+    return declaredPools;
+  }
+
 }

+ 32 - 0
hadoop-mapreduce-project/src/contrib/fairscheduler/src/java/org/apache/hadoop/mapred/UndeclaredPoolException.java

@@ -0,0 +1,32 @@
+/**
+ * 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
+ */
+
+package org.apache.hadoop.mapred;
+
+import java.io.IOException;
+
+/**
+ * Thrown when the pool is not declared in the fair scheduler allocation file.
+ */
+public class UndeclaredPoolException extends IOException {
+
+  private static final long serialVersionUID = -3559057276650280117L;
+
+  public UndeclaredPoolException(String message) {
+    super(message);
+  }
+}

+ 182 - 0
hadoop-mapreduce-project/src/contrib/fairscheduler/src/test/org/apache/hadoop/mapred/TestFairSchedulerPoolNames.java

@@ -0,0 +1,182 @@
+/**
+ * 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.mapred;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileWriter;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.net.URI;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFairSchedulerPoolNames {
+
+  final static String TEST_DIR = new File(System.getProperty("test.build.data",
+      "build/contrib/streaming/test/data")).getAbsolutePath();
+  final static String ALLOC_FILE = new File(TEST_DIR, "test-pools")
+      .getAbsolutePath();
+
+  private static final String POOL_PROPERTY = "pool";
+  private String namenode;
+  private MiniDFSCluster miniDFSCluster = null;
+  private MiniMRCluster miniMRCluster = null;
+
+  /**
+   * Note that The PoolManager.ALLOW_UNDECLARED_POOLS_KEY property is set to
+   * false. So, the default pool is not added, and only pool names in the
+   * scheduler allocation file are considered valid.
+   */
+  @Before
+  public void setUp() throws Exception {
+    new File(TEST_DIR).mkdirs(); // Make sure data directory exists
+    // Create an allocation file with only one pool defined.
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<pool name=\"poolA\">");
+    out.println("<minMaps>1</minMaps>");
+    out.println("<minReduces>1</minReduces>");
+    out.println("</pool>");
+    out.println("</allocations>");
+    out.close();
+
+    Configuration conf = new Configuration();
+    miniDFSCluster = new MiniDFSCluster(conf, 1, true, null);
+    namenode = miniDFSCluster.getFileSystem().getUri().toString();
+
+    JobConf clusterConf = new JobConf();
+    clusterConf.set("mapred.jobtracker.taskScheduler", FairScheduler.class
+        .getName());
+    clusterConf.set("mapred.fairscheduler.allocation.file", ALLOC_FILE);
+    clusterConf.set("mapred.fairscheduler.poolnameproperty", POOL_PROPERTY);
+    clusterConf.setBoolean(FairScheduler.ALLOW_UNDECLARED_POOLS_KEY, false);
+    miniMRCluster = new MiniMRCluster(1, namenode, 1, null, null, clusterConf);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (miniDFSCluster != null) {
+      miniDFSCluster.shutdown();
+    }
+    if (miniMRCluster != null) {
+      miniMRCluster.shutdown();
+    }
+  }
+
+  private void submitJob(String pool) throws IOException {
+    JobConf conf = new JobConf();
+    final Path inDir = new Path("/tmp/testing/wc/input");
+    final Path outDir = new Path("/tmp/testing/wc/output");
+    FileSystem fs = FileSystem.get(URI.create(namenode), conf);
+    fs.delete(outDir, true);
+    if (!fs.mkdirs(inDir)) {
+      throw new IOException("Mkdirs failed to create " + inDir.toString());
+    }
+    DataOutputStream file = fs.create(new Path(inDir, "part-00000"));
+    file.writeBytes("Sample text");
+    file.close();
+
+    FileSystem.setDefaultUri(conf, namenode);
+    conf.set("mapred.job.tracker", "localhost:"
+        + miniMRCluster.getJobTrackerPort());
+    conf.setJobName("wordcount");
+    conf.setInputFormat(TextInputFormat.class);
+
+    // the keys are words (strings)
+    conf.setOutputKeyClass(Text.class);
+    // the values are counts (ints)
+    conf.setOutputValueClass(IntWritable.class);
+
+    conf.setMapperClass(WordCount.MapClass.class);
+    conf.setCombinerClass(WordCount.Reduce.class);
+    conf.setReducerClass(WordCount.Reduce.class);
+
+    FileInputFormat.setInputPaths(conf, inDir);
+    FileOutputFormat.setOutputPath(conf, outDir);
+    conf.setNumMapTasks(1);
+    conf.setNumReduceTasks(0);
+
+    if (pool != null) {
+      conf.set(POOL_PROPERTY, pool);
+    }
+
+    JobClient.runJob(conf);
+  }
+
+  /**
+   * Tests job submission using the default pool name.
+   */
+  @Test
+  public void testDefaultPoolName() {
+    Throwable t = null;
+    try {
+      submitJob(null);
+    } catch (Exception e) {
+      t = e;
+    }
+    assertNotNull("No exception during submission", t);
+    assertTrue("Incorrect exception message", t.getMessage().contains(
+        "Add pool name to the fair scheduler allocation file"));
+  }
+
+  /**
+   * Tests job submission using a valid pool name (i.e., name exists in the fair
+   * scheduler allocation file).
+   */
+  @Test
+  public void testValidPoolName() {
+    Throwable t = null;
+    try {
+      submitJob("poolA");
+    } catch (Exception e) {
+      t = e;
+    }
+    assertNull("Exception during submission", t);
+  }
+
+  /**
+   * Tests job submission using an invalid pool name (i.e., name doesn't exist
+   * in the fair scheduler allocation file).
+   */
+  @Test
+  public void testInvalidPoolName() {
+    Throwable t = null;
+    try {
+      submitJob("poolB");
+    } catch (Exception e) {
+      t = e;
+    }
+    assertNotNull("No exception during submission", t);
+    assertTrue("Incorrect exception message", t.getMessage().contains(
+        "Add pool name to the fair scheduler allocation file"));
+  }
+
+}

+ 9 - 0
hadoop-mapreduce-project/src/docs/src/documentation/content/xdocs/fair_scheduler.xml

@@ -192,6 +192,15 @@
           </td>
           </tr>
           <tr>
+          <td>
+            mapred.fairscheduler.allow.undeclared.pools
+          </td>
+          <td>
+            Boolean property for enabling job submission to pools not declared
+            in the allocation file. Default: true.
+          </td>
+          </tr>
+          <tr>
           <td>
             mapred.fairscheduler.allocation.file
           </td>

+ 1 - 1
hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraChecksum.java

@@ -74,7 +74,7 @@ public class TeraChecksum extends Configured implements Tool {
   }
 
   public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(new Cluster(getConf()), getConf());
+    Job job = Job.getInstance(getConf());
     if (args.length != 2) {
       usage();
       return 2;

+ 1 - 1
hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraGen.java

@@ -280,7 +280,7 @@ public class TeraGen extends Configured implements Tool {
    */
   public int run(String[] args) 
       throws IOException, InterruptedException, ClassNotFoundException {
-    Job job = Job.getInstance(new Cluster(getConf()), getConf());
+    Job job = Job.getInstance(getConf());
     if (args.length != 2) {
       usage();
       return 2;

+ 1 - 1
hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraSort.java

@@ -280,7 +280,7 @@ public class TeraSort extends Configured implements Tool {
 
   public int run(String[] args) throws Exception {
     LOG.info("starting");
-    Job job = Job.getInstance(new Cluster(getConf()), getConf());
+    Job job = Job.getInstance(getConf());
     Path inputDir = new Path(args[0]);
     Path outputDir = new Path(args[1]);
     boolean useSimplePartitioner = getUseSimplePartitioner(job);

+ 1 - 1
hadoop-mapreduce-project/src/examples/org/apache/hadoop/examples/terasort/TeraValidate.java

@@ -157,7 +157,7 @@ public class TeraValidate extends Configured implements Tool {
   }
 
   public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(new Cluster(getConf()), getConf());
+    Job job = Job.getInstance(getConf());
     if (args.length != 2) {
       usage();
       return 1;

+ 7 - 0
hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/JobTracker.java

@@ -3161,6 +3161,13 @@ public class JobTracker implements MRConstants, InterTrackerProtocol,
         throw ace;
       }
 
+      try {
+        this.taskScheduler.checkJobSubmission(job);
+      } catch (IOException ioe){
+        LOG.error("Problem in submitting job " + jobId, ioe);
+        throw ioe;
+      }
+
       // Check the job if it cannot run in the cluster because of invalid memory
       // requirements.
       try {

+ 10 - 0
hadoop-mapreduce-project/src/java/org/apache/hadoop/mapred/TaskScheduler.java

@@ -104,4 +104,14 @@ abstract class TaskScheduler implements Configurable {
   QueueRefresher getQueueRefresher() {
     return null;
   }
+
+  /**
+   * Subclasses can override to provide any scheduler-specific checking
+   * mechanism for job submission.
+   * @param job
+   * @throws IOException
+   */
+  public void checkJobSubmission(JobInProgress job) throws IOException{
+  }
+
 }

+ 1 - 1
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestClusterStatus.java

@@ -208,7 +208,7 @@ public class TestClusterStatus extends TestCase {
     Configuration conf = mr.createJobConf();
     conf.setInt(JobContext.NUM_MAPS, 1);
 
-    Job job = Job.getInstance(cluster, conf);
+    Job job = Job.getInstance(conf);
     job.setNumReduceTasks(1);
     job.setSpeculativeExecution(false);
     job.setJobSetupCleanupNeeded(false);

+ 1 - 1
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapred/TestJobCounters.java

@@ -199,7 +199,7 @@ public class TestJobCounters {
 
   public static Job createJob() throws IOException {
     final Configuration conf = new Configuration();
-    final Job baseJob = Job.getInstance(new Cluster(conf), conf);
+    final Job baseJob = Job.getInstance(conf);
     baseJob.setOutputKeyClass(Text.class);
     baseJob.setOutputValueClass(IntWritable.class);
     baseJob.setMapperClass(NewMapTokenizer.class);

+ 5 - 5
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/TestMapCollection.java

@@ -298,7 +298,7 @@ public class TestMapCollection {
       throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 100);
-    Job job = Job.getInstance(new Cluster(conf), conf);
+    Job job = Job.getInstance(conf);
     conf = job.getConfiguration();
     conf.setInt(MRJobConfig.IO_SORT_MB, ioSortMB);
     conf.set(MRJobConfig.MAP_SORT_SPILL_PERCENT, Float.toString(spillPer));
@@ -409,7 +409,7 @@ public class TestMapCollection {
     // no writes into the serialization buffer
     Configuration conf = new Configuration();
     conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 100);
-    Job job = Job.getInstance(new Cluster(conf), conf);
+    Job job = Job.getInstance(conf);
     conf = job.getConfiguration();
     conf.setInt(MRJobConfig.IO_SORT_MB, 1);
     // 2^20 * spill = 14336 bytes available post-spill, at most 896 meta
@@ -427,7 +427,7 @@ public class TestMapCollection {
   public void testLargeRecConcurrent() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 100);
-    Job job = Job.getInstance(new Cluster(conf), conf);
+    Job job = Job.getInstance(conf);
     conf = job.getConfiguration();
     conf.setInt(MRJobConfig.IO_SORT_MB, 1);
     conf.set(MRJobConfig.MAP_SORT_SPILL_PERCENT, Float.toString(.986328125f));
@@ -496,7 +496,7 @@ public class TestMapCollection {
   public void testRandom() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 100);
-    Job job = Job.getInstance(new Cluster(conf), conf);
+    Job job = Job.getInstance(conf);
     conf = job.getConfiguration();
     conf.setInt(MRJobConfig.IO_SORT_MB, 1);
     conf.setClass("test.mapcollection.class", RandomFactory.class,
@@ -517,7 +517,7 @@ public class TestMapCollection {
   public void testRandomCompress() throws Exception {
     Configuration conf = new Configuration();
     conf.setInt(Job.COMPLETION_POLL_INTERVAL_KEY, 100);
-    Job job = Job.getInstance(new Cluster(conf), conf);
+    Job job = Job.getInstance(conf);
     conf = job.getConfiguration();
     conf.setInt(MRJobConfig.IO_SORT_MB, 1);
     conf.setBoolean(MRJobConfig.MAP_OUTPUT_COMPRESS, true);

+ 3 - 5
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/mapreduce/filecache/TestTrackerDistributedCacheManager.java

@@ -234,12 +234,11 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     }
     TrackerDistributedCacheManager manager = 
       new FakeTrackerDistributedCacheManager(conf);
-    Cluster cluster = new Cluster(conf);
     String userName = getJobOwnerName();
     File workDir = new File(new Path(TEST_ROOT_DIR, "workdir").toString());
 
     // Configures a job with a regular file
-    Job job1 = Job.getInstance(cluster, conf);
+    Job job1 = Job.getInstance(conf);
     job1.setUser(userName);
     job1.addCacheFile(secondCacheFile.toUri());
     Configuration conf1 = job1.getConfiguration();
@@ -262,7 +261,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
     createPrivateTempFile(thirdCacheFile);
     
     // Configures another job with three regular files.
-    Job job2 = Job.getInstance(cluster, conf);
+    Job job2 = Job.getInstance(conf);
     job2.setUser(userName);
     // add a file that would get failed to localize
     job2.addCacheFile(firstCacheFile.toUri());
@@ -366,7 +365,6 @@ public class TestTrackerDistributedCacheManager extends TestCase {
   throws IOException, LoginException, InterruptedException {
     TrackerDistributedCacheManager manager = 
       new TrackerDistributedCacheManager(conf, taskController);
-    Cluster cluster = new Cluster(conf);
     String userName = getJobOwnerName();
     File workDir = new File(TEST_ROOT_DIR, "workdir");
     Path cacheFile = new Path(TEST_ROOT_DIR, "fourthcachefile");
@@ -376,7 +374,7 @@ public class TestTrackerDistributedCacheManager extends TestCase {
       createPrivateTempFile(cacheFile);
     }
     
-    Job job1 = Job.getInstance(cluster, conf);
+    Job job1 = Job.getInstance(conf);
     job1.setUser(userName);
     job1.addCacheFile(cacheFile.toUri());
     Configuration conf1 = job1.getConfiguration();