瀏覽代碼

Merge trunk into HA branch

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1171315 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 年之前
父節點
當前提交
6af0c27930
共有 87 個文件被更改,包括 2690 次插入1650 次删除
  1. 8 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 14 8
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java
  3. 8 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java
  4. 3 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java
  5. 8 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java
  6. 21 1
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/capacity-scheduler.xml
  7. 18 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/commons-logging.properties
  8. 20 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml
  9. 18 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-env.sh
  10. 21 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml
  11. 21 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hdfs-site.xml
  12. 21 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-queue-acls.xml
  13. 21 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-site.xml
  14. 18 0
      hadoop-common-project/hadoop-common/src/main/packages/templates/conf/taskcontroller.cfg
  15. 5 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java
  16. 126 13
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java
  17. 6 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  18. 11 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  19. 5 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
  20. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  21. 2 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
  23. 76 52
      hadoop-mapreduce-project/CHANGES.txt
  24. 5 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  25. 12 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java
  26. 25 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/TaskAttempt.java
  27. 18 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  28. 24 115
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  29. 6 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  30. 110 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
  31. 24 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  32. 22 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  33. 34 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAMWebApp.java
  34. 14 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptReport.java
  35. 25 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskAttemptReportPBImpl.java
  36. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskReportPBImpl.java
  37. 129 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRApps.java
  38. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto
  39. 44 44
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  40. 0 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptUnsuccessfulCompletionEvent.java
  41. 38 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
  42. 12 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java
  43. 118 195
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
  44. 7 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
  45. 12 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java
  46. 97 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java
  47. 99 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java
  48. 25 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsController.java
  49. 69 77
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
  50. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java
  51. 195 35
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java
  52. 235 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
  53. 48 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java
  54. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
  55. 55 7
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java
  56. 15 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientCache.java
  57. 53 170
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  58. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo
  59. 15 203
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
  60. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
  61. 43 469
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
  62. 16 19
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java
  63. 6 12
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  64. 6 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  65. 45 14
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
  66. 9 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java
  67. 247 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java
  68. 15 5
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/test/WebAppTests.java
  69. 11 11
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
  70. 3 3
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  71. 16 8
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
  72. 7 6
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  73. 23 11
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  74. 0 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManagerSubmitEvent.java
  75. 7 10
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
  76. 23 25
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  77. 3 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
  78. 2 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
  79. 6 10
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
  80. 16 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
  81. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
  82. 7 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  83. 15 25
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  84. 1 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java
  85. 101 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java
  86. 10 10
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java
  87. 3 1
      hadoop-mapreduce-project/src/webapps/job/queueinfo.jsp

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

@@ -11,6 +11,9 @@ Trunk (unreleased changes)
 
     HADOOP-7607. Simplify the RPC proxy cleanup process. (atm)
 
+    HADOOP-7635. RetryInvocationHandler should release underlying resources on
+                 close (atm)
+
   BUGS
 
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
@@ -18,6 +21,8 @@ Trunk (unreleased changes)
 
     HADOOP-7610. Fix for hadoop debian package (Eric Yang via gkesavan)
 
+    HADOOP-7641. Add Apache License to template config files (Eric Yang via atm)
+
 Release 0.23.0 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -583,6 +588,9 @@ Release 0.23.0 - Unreleased
 
     HADOOP-7626. Bugfix for a config generator (Eric Yang via ddas)
 
+    HADOOP-7629. Allow immutable FsPermission objects to be used as IPC
+    parameters. (todd)
+
 Release 0.22.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 14 - 8
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java

@@ -44,18 +44,12 @@ public class FsPermission implements Writable {
   };
   static {                                      // register a ctor
     WritableFactories.setFactory(FsPermission.class, FACTORY);
+    WritableFactories.setFactory(ImmutableFsPermission.class, FACTORY);
   }
 
   /** Create an immutable {@link FsPermission} object. */
   public static FsPermission createImmutable(short permission) {
-    return new FsPermission(permission) {
-      public FsPermission applyUMask(FsPermission umask) {
-        throw new UnsupportedOperationException();
-      }
-      public void readFields(DataInput in) throws IOException {
-        throw new UnsupportedOperationException();
-      }
-    };
+    return new ImmutableFsPermission(permission);
   }
 
   //POSIX permission style
@@ -301,4 +295,16 @@ public class FsPermission implements Writable {
 
     return new FsPermission((short)n);
   }
+  
+  private static class ImmutableFsPermission extends FsPermission {
+    public ImmutableFsPermission(short permission) {
+      super(permission);
+    }
+    public FsPermission applyUMask(FsPermission umask) {
+      throw new UnsupportedOperationException();
+    }
+    public void readFields(DataInput in) throws IOException {
+      throw new UnsupportedOperationException();
+    }    
+  }
 }

+ 8 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/DefaultFailoverProxyProvider.java

@@ -17,7 +17,10 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.RPC;
 
 /**
  * An implementation of {@link FailoverProxyProvider} which does nothing in the
@@ -49,4 +52,9 @@ public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
     // Nothing to do.
   }
 
+  @Override
+  public void close() throws IOException {
+    RPC.stopProxy(proxy);
+  }
+
 }

+ 3 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/FailoverProxyProvider.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.Closeable;
+
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
@@ -27,7 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  * {@link RetryPolicy}.
  */
 @InterfaceStability.Evolving
-public interface FailoverProxyProvider {
+public interface FailoverProxyProvider extends Closeable {
 
   /**
    * Get the proxy object which should be used until the next failover event

+ 8 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryInvocationHandler.java

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.retry;
 
+import java.io.Closeable;
+import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -27,7 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 
-class RetryInvocationHandler implements InvocationHandler {
+class RetryInvocationHandler implements InvocationHandler, Closeable {
   public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
   private FailoverProxyProvider proxyProvider;
   
@@ -103,4 +105,9 @@ class RetryInvocationHandler implements InvocationHandler {
     }
   }
 
+  @Override
+  public void close() throws IOException {
+    proxyProvider.close();
+  }
+
 }

+ 21 - 1
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/capacity-scheduler.xml

@@ -1,5 +1,25 @@
 <?xml version="1.0"?>
-
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 <!-- This is the configuration file for the resource manager in Hadoop. -->
 <!-- You can configure various scheduling parameters related to queues. -->
 <!-- The properties for a queue follow a naming convention,such as, -->

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/commons-logging.properties

@@ -1,3 +1,21 @@
+# Copyright 2011 The Apache Software Foundation
+# 
+# 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.
+
 #Logging Implementation
 
 #Log4J

+ 20 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/core-site.xml

@@ -1,6 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
 
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 <!-- Put site-specific property overrides in this file. -->
 
 <configuration>

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-env.sh

@@ -1,3 +1,21 @@
+# Copyright 2011 The Apache Software Foundation
+# 
+# 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.
+
 # Set Hadoop-specific environment variables here.
 
 # The only required environment variable is JAVA_HOME.  All others are

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hadoop-policy.xml

@@ -1,5 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 
 <!-- Put site-specific property overrides in this file. -->
 

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/hdfs-site.xml

@@ -1,5 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 
 <configuration>
 

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/mapred-queue-acls.xml

@@ -1,5 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 <configuration>
 <property>
 <name>mapred.queue.default.acl-submit-job</name>

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

@@ -1,5 +1,26 @@
 <?xml version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+
+ Copyright 2011 The Apache Software Foundation
+ 
+ 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.
+
+-->
 
 <!-- Put site-specific property overrides in this file. -->
 

+ 18 - 0
hadoop-common-project/hadoop-common/src/main/packages/templates/conf/taskcontroller.cfg

@@ -1,3 +1,21 @@
+# Copyright 2011 The Apache Software Foundation
+# 
+# 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.
+
 mapreduce.cluster.local.dir=${HADOOP_MAPRED_DIR}
 mapreduce.tasktracker.group=${HADOOP_GROUP}
 hadoop.log.dir=${HADOOP_LOG_DIR}/${HADOOP_MR_USER}

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/retry/TestFailoverProxy.java

@@ -57,6 +57,11 @@ public class TestFailoverProxy {
     public Class<?> getInterface() {
       return iface;
     }
+
+    @Override
+    public void close() throws IOException {
+      // Nothing to do.
+    }
     
   }
   

+ 126 - 13
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java

@@ -18,28 +18,38 @@
 
 package org.apache.hadoop.ipc;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.util.Arrays;
 
-import junit.framework.TestCase;
+import javax.net.SocketFactory;
 
 import org.apache.commons.logging.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
+import org.apache.hadoop.security.token.SecretManager;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Test;
+import static org.junit.Assert.*;
 
 import com.google.protobuf.DescriptorProtos;
 import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
@@ -49,18 +59,22 @@ import static org.apache.hadoop.test.MetricsAsserts.*;
 import static org.mockito.Mockito.*;
 
 /** Unit tests for RPC. */
-public class TestRPC extends TestCase {
+@SuppressWarnings("deprecation")
+public class TestRPC {
   private static final String ADDRESS = "0.0.0.0";
 
   public static final Log LOG =
     LogFactory.getLog(TestRPC.class);
   
   private static Configuration conf = new Configuration();
+  
+  static {
+    conf.setClass("rpc.engine." + StoppedProtocol.class.getName(),
+        StoppedRpcEngine.class, RpcEngine.class);
+  }
 
   int datasize = 1024*100;
   int numThreads = 50;
-
-  public TestRPC(String name) { super(name); }
 	
   public interface TestProtocol extends VersionedProtocol {
     public static final long versionID = 1L;
@@ -207,6 +221,74 @@ public class TestRPC extends TestCase {
     }
   }
   
+  /**
+   * A basic interface for testing client-side RPC resource cleanup.
+   */
+  private static interface StoppedProtocol {
+    long versionID = 0;
+
+    public void stop();
+  }
+  
+  /**
+   * A class used for testing cleanup of client side RPC resources.
+   */
+  private static class StoppedRpcEngine implements RpcEngine {
+
+    @Override
+    public Object[] call(Method method, Object[][] params, InetSocketAddress[] addrs,
+        UserGroupInformation ticket, Configuration conf)
+        throws IOException, InterruptedException {
+      return null;
+    }
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
+        InetSocketAddress addr, UserGroupInformation ticket, Configuration conf,
+        SocketFactory factory, int rpcTimeout) throws IOException {
+      T proxy = (T) Proxy.newProxyInstance(protocol.getClassLoader(),
+              new Class[] { protocol }, new StoppedInvocationHandler());
+      return new ProtocolProxy<T>(protocol, proxy, false);
+    }
+
+    @Override
+    public org.apache.hadoop.ipc.RPC.Server getServer(Class<?> protocol,
+        Object instance, String bindAddress, int port, int numHandlers,
+        int numReaders, int queueSizePerHandler, boolean verbose, Configuration conf,
+        SecretManager<? extends TokenIdentifier> secretManager) throws IOException {
+      return null;
+    }
+    
+  }
+
+  /**
+   * An invocation handler which does nothing when invoking methods, and just
+   * counts the number of times close() is called.
+   */
+  private static class StoppedInvocationHandler
+      implements InvocationHandler, Closeable {
+    
+    private int closeCalled = 0;
+
+    @Override
+    public Object invoke(Object proxy, Method method, Object[] args)
+        throws Throwable {
+          return null;
+    }
+
+    @Override
+    public void close() throws IOException {
+      closeCalled++;
+    }
+    
+    public int getCloseCalled() {
+      return closeCalled;
+    }
+    
+  }
+  
+  @Test
   public void testConfRpc() throws Exception {
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, 1, false, conf, null);
@@ -229,6 +311,7 @@ public class TestRPC extends TestCase {
     server.stop();    
   }
 
+  @Test
   public void testSlowRpc() throws Exception {
     System.out.println("Testing Slow RPC");
     // create a server with two handlers
@@ -273,11 +356,12 @@ public class TestRPC extends TestCase {
     }
   }
   
-  public void testRPCConf(Configuration conf) throws Exception {
-    
+  @Test
+  public void testCalls() throws Exception {
+    testCallsInternal(conf);
   }
-
-  public void testCalls(Configuration conf) throws Exception {
+  
+  private void testCallsInternal(Configuration conf) throws Exception {
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, conf);
     TestProtocol proxy = null;
@@ -384,6 +468,7 @@ public class TestRPC extends TestCase {
     }
   }
   
+  @Test
   public void testStandaloneClient() throws IOException {
     try {
       TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
@@ -450,6 +535,7 @@ public class TestRPC extends TestCase {
     }
   }
   
+  @Test
   public void testAuthorization() throws Exception {
     Configuration conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
@@ -481,20 +567,48 @@ public class TestRPC extends TestCase {
     Configuration conf = new Configuration();
     
     conf.setBoolean("ipc.client.ping", false);
-    new TestRPC("testnoPings").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
     
     conf.setInt(CommonConfigurationKeys.IPC_SERVER_RPC_READ_THREADS_KEY, 2);
-    new TestRPC("testnoPings").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
   }
 
   /**
    * Test stopping a non-registered proxy
    * @throws Exception
    */
+  @Test
   public void testStopNonRegisteredProxy() throws Exception {
     RPC.stopProxy(mock(TestProtocol.class));
   }
   
+  @Test
+  public void testStopProxy() throws IOException {
+    StoppedProtocol proxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
+        StoppedProtocol.versionID, null, conf);
+    StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
+        Proxy.getInvocationHandler(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 0);
+    RPC.stopProxy(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 1);
+  }
+  
+  @Test
+  public void testWrappedStopProxy() throws IOException {
+    StoppedProtocol wrappedProxy = (StoppedProtocol) RPC.getProxy(StoppedProtocol.class,
+        StoppedProtocol.versionID, null, conf);
+    StoppedInvocationHandler invocationHandler = (StoppedInvocationHandler)
+        Proxy.getInvocationHandler(wrappedProxy);
+    
+    StoppedProtocol proxy = (StoppedProtocol) RetryProxy.create(StoppedProtocol.class,
+        wrappedProxy, RetryPolicies.RETRY_FOREVER);
+    
+    assertEquals(invocationHandler.getCloseCalled(), 0);
+    RPC.stopProxy(proxy);
+    assertEquals(invocationHandler.getCloseCalled(), 1);
+  }
+  
+  @Test
   public void testErrorMsgForInsecureClient() throws Exception {
     final Server server = RPC.getServer(TestProtocol.class,
         new TestImpl(), ADDRESS, 0, 5, true, conf, null);
@@ -567,10 +681,10 @@ public class TestRPC extends TestCase {
     return count;
   }
 
-
   /**
    * Test that server.stop() properly stops all threads
    */
+  @Test
   public void testStopsAllThreads() throws Exception {
     int threadsBefore = countThreads("Server$Listener$Reader");
     assertEquals("Expect no Reader threads running before test",
@@ -591,8 +705,7 @@ public class TestRPC extends TestCase {
   }
   
   public static void main(String[] args) throws Exception {
-
-    new TestRPC("test").testCalls(conf);
+    new TestRPC().testCallsInternal(conf);
 
   }
 }

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

@@ -35,6 +35,12 @@ Trunk (unreleased changes)
 
     HDFS-2314. MRV1 test compilation broken after HDFS-2197 (todd)
 
+    HDFS-2331. Fix WebHdfsFileSystem compilation problems for a bug in JDK
+    version < 1.6.0_26.  (Abhijit Suresh Shingate via szetszwo)
+
+    HDFS-2333. Change DFSOutputStream back to package private, otherwise,
+    there are two SC_START_IN_CTOR findbugs warnings.  (szetszwo)
+
 Release 0.23.0 - Unreleased
 
   INCOMPATIBLE CHANGES

+ 11 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
@@ -79,7 +80,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -797,12 +797,20 @@ public class DFSClient implements java.io.Closeable {
    * 
    * @param src file name
    * @param buffersize buffer size
-   * @param progress for reporting write-progress
+   * @param progress for reporting write-progress; null is acceptable.
+   * @param statistics file system statistics; null is acceptable.
    * @return an output stream for writing into the file
    * 
    * @see ClientProtocol#append(String, String) 
    */
-  public DFSOutputStream append(String src, int buffersize, Progressable progress) 
+  public FSDataOutputStream append(final String src, final int buffersize,
+      final Progressable progress, final FileSystem.Statistics statistics
+      ) throws IOException {
+    final DFSOutputStream out = append(src, buffersize, progress);
+    return new FSDataOutputStream(out, statistics, out.getInitialLen());
+  }
+
+  private DFSOutputStream append(String src, int buffersize, Progressable progress) 
       throws IOException {
     checkOpen();
     HdfsFileStatus stat = getFileInfo(src);

+ 5 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -36,6 +36,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -98,7 +99,8 @@ import org.apache.hadoop.util.PureJavaCrc32;
  * datanode from the original pipeline. The DataStreamer now
  * starts sending packets from the dataQueue.
 ****************************************************************/
-public class DFSOutputStream extends FSOutputSummer implements Syncable {
+@InterfaceAudience.Private
+class DFSOutputStream extends FSOutputSummer implements Syncable {
   private final DFSClient dfsClient;
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private Socket s;
@@ -1537,7 +1539,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
    * write pipeline have failed. 
    * @return the number of valid replicas of the current block
    */
-  public synchronized int getNumCurrentReplicas() throws IOException {
+  synchronized int getNumCurrentReplicas() throws IOException {
     dfsClient.checkOpen();
     isClosed();
     if (streamer == null) {
@@ -1707,7 +1709,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
   /**
    * Returns the size of a file as it was when this stream was opened
    */
-  public long getInitialLen() {
+  long getInitialLen() {
     return initialFileSize;
   }
 

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

@@ -240,10 +240,8 @@ public class DistributedFileSystem extends FileSystem {
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
-
     statistics.incrementWriteOps(1);
-    final DFSOutputStream op = dfs.append(getPathName(f), bufferSize, progress);
-    return new FSDataOutputStream(op, statistics, op.getInitialLen());
+    return dfs.append(getPathName(f), bufferSize, progress, statistics);
   }
 
   @Override

+ 2 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/resources/DatanodeWebHdfsMethods.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
-import org.apache.hadoop.hdfs.DFSOutputStream;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -158,10 +157,8 @@ public class DatanodeWebHdfsMethods {
       final Configuration conf = new Configuration(datanode.getConf());
       final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
       final DFSClient dfsclient = new DFSClient(nnRpcAddr, conf);
-      final DFSOutputStream dfsout = dfsclient.append(fullpath,
-          bufferSize.getValue(), null);
-      final FSDataOutputStream out = new FSDataOutputStream(dfsout, null,
-          dfsout.getInitialLen());
+      final FSDataOutputStream out = dfsclient.append(fullpath,
+          bufferSize.getValue(), null, null);
       try {
         IOUtils.copyBytes(in, out, bufferSize.getValue());
       } finally {

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

@@ -206,7 +206,7 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     final HttpURLConnection conn = httpConnect(op, fspath, parameters);
     validateResponse(op, conn);
     try {
-      return jsonParse(conn.getInputStream());
+      return WebHdfsFileSystem.<T>jsonParse(conn.getInputStream());
     } finally {
       conn.disconnect();
     }

+ 76 - 52
hadoop-mapreduce-project/CHANGES.txt

@@ -286,6 +286,18 @@ Release 0.23.0 - Unreleased
     org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to
     all public records. (acmurthy)
 
+    MAPREDUCE-2676. MR-279: JobHistory Job page needs reformatted. (Robert Evans via 
+    mahadev)
+
+    MAPREDUCE-2899. Replace major parts of ApplicationSubmissionContext with a 
+    ContainerLaunchContext (Arun Murthy via mahadev)
+
+    MAPREDUCE-2966. Added ShutDown hooks for MRV2 processes so that they can
+    gracefully exit. (Abhijit Suresh Shingate via vinodkv)
+
+    MAPREDUCE-2672. MR-279: JobHistory Server needs Analysis this job. 
+    (Robert Evans via mahadev)
+
   OPTIMIZATIONS
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
@@ -1234,83 +1246,92 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2897. Javadoc for ClientRMProtocol protocol and related records. 
     (acmurthy)
 
-   MAPREDUCE-2916. Ivy build for MRv1 fails with bad organization for 
-   common daemon. (mahadev)
-
-   MAPREDUCE-2917. Fixed corner case in container reservation which led to
-   starvation and hung jobs. (acmurthy) 
+    MAPREDUCE-2916. Ivy build for MRv1 fails with bad organization for 
+    common daemon. (mahadev)
 
-   MAPREDUCE-2756. Better error handling in JobControl for failed jobs.
-   (Robert Evans via acmurthy) 
+    MAPREDUCE-2917. Fixed corner case in container reservation which led to
+    starvation and hung jobs. (acmurthy) 
 
-   MAPREDUCE-2716. MRReliabilityTest job fails because of missing
-   job-file. (Jeffrey Naisbitt via vinodkv)
+    MAPREDUCE-2756. Better error handling in JobControl for failed jobs.
+    (Robert Evans via acmurthy) 
 
-   MAPREDUCE-2882. TestLineRecordReader depends on ant jars. (todd)
+    MAPREDUCE-2716. MRReliabilityTest job fails because of missing
+    job-file. (Jeffrey Naisbitt via vinodkv)
 
-   MAPREDUCE-2687. Fix NodeManager to use the right version of
-   LocalDirAllocator.getLocalPathToWrite. (mahadev & acmurthy) 
+    MAPREDUCE-2882. TestLineRecordReader depends on ant jars. (todd)
 
-   MAPREDUCE-2800. Set final progress for tasks to ensure all task information
-   is correctly logged to JobHistory. (Siddharth Seth via acmurthy)
+    MAPREDUCE-2687. Fix NodeManager to use the right version of
+    LocalDirAllocator.getLocalPathToWrite. (mahadev & acmurthy) 
 
-   MAPREDUCE-2938. Log application submission failure in CapacityScheduler.
-   (acmurthy) 
+    MAPREDUCE-2800. Set final progress for tasks to ensure all task information
+    is correctly logged to JobHistory. (Siddharth Seth via acmurthy)
 
-   MAPREDUCE-2948. Hadoop streaming test failure, post MR-2767 (mahadev)
+    MAPREDUCE-2938. Log application submission failure in CapacityScheduler.
+    (acmurthy) 
 
-   MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy) 
+    MAPREDUCE-2948. Hadoop streaming test failure, post MR-2767 (mahadev)
 
-   MAPREDUCE-2942. TestNMAuditLogger.testNMAuditLoggerWithIP failing (Thomas Graves 
-   via mahadev)
+    MAPREDUCE-2908. Fix all findbugs warnings. (vinodkv via acmurthy) 
 
-   MAPREDUCE-2947. Fixed race condition in AuxiliaryServices. (vinodkv via
-   acmurthy) 
+    MAPREDUCE-2942. TestNMAuditLogger.testNMAuditLoggerWithIP failing (Thomas Graves 
+    via mahadev)
 
-   MAPREDUCE-2844. Fixed display of nodes in UI. (Ravi Teja Ch N V via
-   acmurthy) 
+    MAPREDUCE-2947. Fixed race condition in AuxiliaryServices. (vinodkv via
+    acmurthy) 
 
-   MAPREDUCE-2677. Fixed 404 for some links from HistoryServer. (Robert Evans
-   via acmurthy) 
+    MAPREDUCE-2844. Fixed display of nodes in UI. (Ravi Teja Ch N V via
+    acmurthy) 
 
-   MAPREDUCE-2937. Ensure reason for application failure is displayed to the
-   user. (mahadev via acmurthy) 
+    MAPREDUCE-2677. Fixed 404 for some links from HistoryServer. (Robert Evans
+    via acmurthy) 
 
-   MAPREDUCE-2953. Fix a race condition on submission which caused client to 
-   incorrectly assume application was gone by making submission synchronous
-   for RMAppManager. (Thomas Graves via acmurthy) 
+    MAPREDUCE-2937. Ensure reason for application failure is displayed to the
+    user. (mahadev via acmurthy) 
 
-   MAPREDUCE-2963. Fix hang in TestMRJobs. (Siddharth Seth via acmurthy) 
+    MAPREDUCE-2953. Fix a race condition on submission which caused client to 
+    incorrectly assume application was gone by making submission synchronous
+    for RMAppManager. (Thomas Graves via acmurthy) 
 
-   MAPREDUCE-2954. Fixed a deadlock in NM caused due to wrong synchronization
-   in protocol buffer records. (Siddharth Seth via vinodkv)
+    MAPREDUCE-2963. Fix hang in TestMRJobs. (Siddharth Seth via acmurthy) 
+ 
+    MAPREDUCE-2954. Fixed a deadlock in NM caused due to wrong synchronization
+    in protocol buffer records. (Siddharth Seth via vinodkv)
 
-   MAPREDUCE-2975. Fixed YARNRunner to use YarnConfiguration rather than
-   Configuration. (mahadev via acmurthy) 
+    MAPREDUCE-2975. Fixed YARNRunner to use YarnConfiguration rather than
+    Configuration. (mahadev via acmurthy) 
  
-   MAPREDUCE-2971. ant build mapreduce fails protected access jc.displayJobList
-   (jobs) (Thomas Graves via mahadev)
+    MAPREDUCE-2971. ant build mapreduce fails protected access jc.displayJobList
+    (jobs) (Thomas Graves via mahadev)
 
-   MAPREDUCE-2691. Finishing up the cleanup of distributed cache file resources
-   and related tests. (Siddharth Seth via vinodkv)
+    MAPREDUCE-2691. Finishing up the cleanup of distributed cache file resources
+    and related tests. (Siddharth Seth via vinodkv)
 
-   MAPREDUCE-2749. Ensure NM registers with RM after starting all its services
-   correctly. (Thomas Graves via acmurthy)
+    MAPREDUCE-2749. Ensure NM registers with RM after starting all its services
+    correctly. (Thomas Graves via acmurthy)
 
-   MAPREDUCE-2979. Removed the needless ClientProtocolProvider configuration
-   from the hadoop-mapreduce-client-core module. (Siddharth Seth via vinodkv)
+    MAPREDUCE-2979. Removed the needless ClientProtocolProvider configuration
+    from the hadoop-mapreduce-client-core module. (Siddharth Seth via vinodkv)
 
-   MAPREDUCE-2985. Fixed findbugs warnings in ResourceLocalizationService.
-   (Thomas Graves via acmurthy)
+    MAPREDUCE-2985. Fixed findbugs warnings in ResourceLocalizationService.
+    (Thomas Graves via acmurthy)
 
-   MAPREDUCE-2874. Fix formatting of ApplicationId in web-ui. (Eric Payne via
-   acmurthy)
+    MAPREDUCE-2874. Fix formatting of ApplicationId in web-ui. (Eric Payne via
+    acmurthy)
 
-   MAPREDUCE-2995. Better handling of expired containers in MapReduce
-   ApplicationMaster. (vinodkv via acmurthy) 
+    MAPREDUCE-2995. Better handling of expired containers in MapReduce
+    ApplicationMaster. (vinodkv via acmurthy) 
 
-   MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via 
-   acmurthy) 
+    MAPREDUCE-2995. Fixed race condition in ContainerLauncher. (vinodkv via 
+    acmurthy) 
+
+    MAPREDUCE-2949. Fixed NodeManager to shut-down correctly if a service
+    startup fails. (Ravi Teja via vinodkv)
+
+    MAPREDUCE-3005. Fix both FifoScheduler and CapacityScheduler to correctly
+    enforce locality constraints. (acmurthy) 
+
+    MAPREDUCE-3007. Fixed Yarn Mapreduce client to be able to connect to 
+    JobHistoryServer in secure mode. (vinodkv)
 
 Release 0.22.0 - Unreleased
 
@@ -1888,6 +1909,9 @@ Release 0.22.0 - Unreleased
 
     MAPREDUCE-2767. Remove Linux task-controller. (Milind Bhandarkar via shv)
 
+    MAPREDUCE-2991. queueinfo.jsp fails to show queue status for Capacity 
+    scheduler if queue names contain special symbols. (Priyo Mustafi via shv)
+
 Release 0.21.1 - Unreleased
 
   NEW FEATURES

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

@@ -559,12 +559,14 @@ public class MRAppMaster extends CompositeService {
   public static void main(String[] args) {
     try {
       //Configuration.addDefaultResource("job.xml");
-      ApplicationId applicationId = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class);
-      
+      ApplicationId applicationId = RecordFactoryProvider
+          .getRecordFactory(null).newRecordInstance(ApplicationId.class);
       applicationId.setClusterTimestamp(Long.valueOf(args[0]));
       applicationId.setId(Integer.valueOf(args[1]));
       int failCount = Integer.valueOf(args[2]);
       MRAppMaster appMaster = new MRAppMaster(applicationId, failCount);
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(appMaster));
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
       conf.addResource(new Path(MRConstants.JOB_CONF_FILE));
       conf.set(MRJobConfig.USER_NAME, 
@@ -573,7 +575,7 @@ public class MRAppMaster extends CompositeService {
       appMaster.init(conf);
       appMaster.start();
     } catch (Throwable t) {
-      LOG.error("Caught throwable. Exiting:", t);
+      LOG.fatal("Error starting MRAppMaster", t);
       System.exit(1);
     }
   } 

+ 12 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/Job.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.job;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -30,6 +31,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptCompletionEvent;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 
 
 /**
@@ -52,6 +54,16 @@ public interface Job {
   int getCompletedReduces();
   boolean isUber();
   String getUserName();
+  
+  /**
+   * @return a path to where the config file for this job is located.
+   */
+  Path getConfFile();
+  
+  /**
+   * @return the ACLs for this job for each type of JobACL given. 
+   */
+  Map<JobACL, AccessControlList> getJobACLs();
 
   TaskAttemptCompletionEvent[]
       getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);

+ 25 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/TaskAttempt.java

@@ -38,31 +38,50 @@ public interface TaskAttempt {
   float getProgress();
   TaskAttemptState getState();
 
-  /** Has attempt reached the final state or not.
+  /** 
+   * Has attempt reached the final state or not.
+   * @return true if it has finished, else false
    */
   boolean isFinished();
 
-  /**If container Assigned then return container ID, otherwise null.
+  /**
+   * @return the container ID if a container is assigned, otherwise null.
    */
   ContainerId getAssignedContainerID();
 
-  /**If container Assigned then return container mgr address, otherwise null.
+  /**
+   * @return container mgr address if a container is assigned, otherwise null.
    */
   String getAssignedContainerMgrAddress();
   
-  /**If container Assigned then return the node's http address, otherwise null.
+  /**
+   * @return node's http address if a container is assigned, otherwise null.
    */
   String getNodeHttpAddress();
 
-  /** Returns time at which container is launched. If container is not launched
+  /** 
+   * @return time at which container is launched. If container is not launched
    * yet, returns 0.
    */
   long getLaunchTime();
 
-  /** Returns attempt's finish time. If attempt is not finished
+  /** 
+   * @return attempt's finish time. If attempt is not finished
    *  yet, returns 0.
    */
   long getFinishTime();
+  
+  /**
+   * @return The attempt's shuffle finish time if the attempt is a reduce. If
+   * attempt is not finished yet, returns 0.
+   */
+  long getShuffleFinishTime();
+
+  /**
+   * @return The attempt's sort or merge finish time if the attempt is a reduce. 
+   * If attempt is not finished yet, returns 0.
+   */
+  long getSortFinishTime();
 
   /**
    * @return the port shuffle is on.

+ 18 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java

@@ -772,6 +772,15 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     return userName;
   }
   
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
+   */
+  @Override
+  public Path getConfFile() {
+    return remoteJobConfFile;
+  }
+  
   @Override
   public String getName() {
     return jobName;
@@ -787,6 +796,15 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   public int getTotalReduces() {
     return reduceTasks.size();  //FIXME: why indirection? return numReduceTasks
   }
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getJobACLs()
+   */
+  @Override
+  public Map<JobACL, AccessControlList> getJobACLs() {
+    return Collections.unmodifiableMap(jobACLs);
+  }
 
   public static class InitTransition 
       implements MultipleArcTransition<JobImpl, JobEvent, JobState> {

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

@@ -579,13 +579,12 @@ public abstract class TaskAttemptImpl implements
           + remoteJobConfPath.toUri().toASCIIString());
       // //////////// End of JobConf setup
 
-      
       // Setup DistributedCache
-      setupDistributedCache(remoteFS, conf, localResources, environment);
+      MRApps.setupDistributedCache(conf, localResources, environment);
 
       // Set local-resources and environment
       container.setLocalResources(localResources);
-      container.setEnv(environment);
+      container.setEnvironment(environment);
       
       // Setup up tokens
       Credentials taskCredentials = new Credentials();
@@ -618,7 +617,7 @@ public abstract class TaskAttemptImpl implements
           ShuffleHandler.serializeServiceData(jobToken));
       container.setServiceData(serviceData);
 
-      MRApps.addToClassPath(container.getEnv(), getInitialClasspath());
+      MRApps.addToClassPath(container.getEnvironment(), getInitialClasspath());
     } catch (IOException e) {
       throw new YarnException(e);
     }
@@ -645,7 +644,7 @@ public abstract class TaskAttemptImpl implements
         taskAttemptListener.getAddress(), remoteTask, javaHome,
         workDir.toString(), containerLogDir, childTmpDir, jvmID));
 
-    MapReduceChildJVM.setVMEnv(container.getEnv(), classPaths,
+    MapReduceChildJVM.setVMEnv(container.getEnvironment(), classPaths,
         workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask,
         localizedApplicationTokensFile);
 
@@ -656,116 +655,6 @@ public abstract class TaskAttemptImpl implements
     return container;
   }
 
-  private static long[] parseTimeStamps(String[] strs) {
-    if (null == strs) {
-      return null;
-    }
-    long[] result = new long[strs.length];
-    for(int i=0; i < strs.length; ++i) {
-      result[i] = Long.parseLong(strs[i]);
-    }
-    return result;
-  }
-
-  private void setupDistributedCache(FileSystem remoteFS, 
-      Configuration conf, 
-      Map<String, LocalResource> localResources,
-      Map<String, String> env) 
-  throws IOException {
-    
-    // Cache archives
-    parseDistributedCacheArtifacts(remoteFS, localResources, env, 
-        LocalResourceType.ARCHIVE, 
-        DistributedCache.getCacheArchives(conf), 
-        parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)), 
-        getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES), 
-        DistributedCache.getArchiveVisibilities(conf), 
-        DistributedCache.getArchiveClassPaths(conf));
-    
-    // Cache files
-    parseDistributedCacheArtifacts(remoteFS, 
-        localResources, env, 
-        LocalResourceType.FILE, 
-        DistributedCache.getCacheFiles(conf),
-        parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
-        getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
-        DistributedCache.getFileVisibilities(conf),
-        DistributedCache.getFileClassPaths(conf));
-  }
-
-  // TODO - Move this to MR!
-  // Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], 
-  // long[], boolean[], Path[], FileType)
-  private void parseDistributedCacheArtifacts(
-      FileSystem remoteFS, 
-      Map<String, LocalResource> localResources,
-      Map<String, String> env,
-      LocalResourceType type,
-      URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], 
-      Path[] pathsToPutOnClasspath) throws IOException {
-
-    if (uris != null) {
-      // Sanity check
-      if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
-          (uris.length != visibilities.length)) {
-        throw new IllegalArgumentException("Invalid specification for " +
-        		"distributed-cache artifacts of type " + type + " :" +
-        		" #uris=" + uris.length +
-        		" #timestamps=" + timestamps.length +
-        		" #visibilities=" + visibilities.length
-        		);
-      }
-      
-      Map<String, Path> classPaths = new HashMap<String, Path>();
-      if (pathsToPutOnClasspath != null) {
-        for (Path p : pathsToPutOnClasspath) {
-          p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
-              remoteFS.getWorkingDirectory()));
-          classPaths.put(p.toUri().getPath().toString(), p);
-        }
-      }
-      for (int i = 0; i < uris.length; ++i) {
-        URI u = uris[i];
-        Path p = new Path(u);
-        p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
-            remoteFS.getWorkingDirectory()));
-        // Add URI fragment or just the filename
-        Path name = new Path((null == u.getFragment())
-          ? p.getName()
-          : u.getFragment());
-        if (name.isAbsolute()) {
-          throw new IllegalArgumentException("Resource name must be relative");
-        }
-        String linkName = name.toUri().getPath();
-        localResources.put(
-            linkName,
-            BuilderUtils.newLocalResource(
-                p.toUri(), type, 
-                visibilities[i]
-                  ? LocalResourceVisibility.PUBLIC
-                  : LocalResourceVisibility.PRIVATE,
-                sizes[i], timestamps[i])
-        );
-        if (classPaths.containsKey(u.getPath())) {
-          MRApps.addToClassPath(env, linkName);
-        }
-      }
-    }
-  }
-  
-  // TODO - Move this to MR!
-  private static long[] getFileSizes(Configuration conf, String key) {
-    String[] strs = conf.getStrings(key);
-    if (strs == null) {
-      return null;
-    }
-    long[] result = new long[strs.length];
-    for(int i=0; i < strs.length; ++i) {
-      result[i] = Long.parseLong(strs[i]);
-    }
-    return result;
-  }
-  
   @Override
   public ContainerId getAssignedContainerID() {
     readLock.lock();
@@ -806,6 +695,25 @@ public abstract class TaskAttemptImpl implements
     }
   }
 
+  @Override
+  public long getShuffleFinishTime() {
+    readLock.lock();
+    try {
+      return this.reportedStatus.shuffleFinishTime;
+    } finally {
+      readLock.unlock();
+    }
+  }
+
+  @Override
+  public long getSortFinishTime() {
+    readLock.lock();
+    try {
+      return this.reportedStatus.sortFinishTime;
+    } finally {
+      readLock.unlock();
+    }
+  }
 
   @Override
   public int getShufflePort() {
@@ -862,6 +770,7 @@ public abstract class TaskAttemptImpl implements
       result.setProgress(reportedStatus.progress);
       result.setStartTime(launchTime);
       result.setFinishTime(finishTime);
+      result.setShuffleFinishTime(this.reportedStatus.shuffleFinishTime);
       result.setDiagnosticInfo(reportedStatus.diagnosticInfo);
       result.setPhase(reportedStatus.phase);
       result.setStateString(reportedStatus.stateString);

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

@@ -177,11 +177,12 @@ public class AppController extends Controller implements AMParams {
         }
         setTitle(join(attemptState, " ",
             MRApps.taskType(taskType).toString(), " attempts in ", $(JOB_ID)));
+
+        render(attemptsPage());
       } catch (Exception e) {
         badRequest(e.getMessage());
       }
     }
-    render(attemptsPage());
   }
 
   /**
@@ -205,7 +206,7 @@ public class AppController extends Controller implements AMParams {
   /**
    * Ensure that a JOB_ID was passed into the page.
    */
-  void requireJob() {
+  public void requireJob() {
     try {
       if ($(JOB_ID).isEmpty()) {
         throw new RuntimeException("missing job ID");
@@ -216,14 +217,15 @@ public class AppController extends Controller implements AMParams {
         notFound($(JOB_ID));
       }
     } catch (Exception e) {
-      badRequest(e.getMessage() == null ? e.getClass().getName() : e.getMessage());
+      badRequest(e.getMessage() == null ? 
+          e.getClass().getName() : e.getMessage());
     }
   }
 
   /**
    * Ensure that a TASK_ID was passed into the page.
    */
-  void requireTask() {
+  public void requireTask() {
     try {
       if ($(TASK_ID).isEmpty()) {
         throw new RuntimeException("missing task ID");

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

@@ -0,0 +1,110 @@
+/**
+* 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.webapp.view.JQueryUI._TH;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+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;
+import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import com.google.inject.Inject;
+
+/**
+ * Render the configuration for this job.
+ */
+public class ConfBlock extends HtmlBlock {
+  final AppContext appContext;
+  final Configuration conf;
+
+  @Inject ConfBlock(AppContext appctx, Configuration conf) {
+    appContext = appctx;
+    this.conf = conf;
+  }
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
+   */
+  @Override protected void render(Block html) {
+    String jid = $(JOB_ID);
+    if (jid.isEmpty()) {
+      html.
+        p()._("Sorry, can't do anything without a JobID.")._();
+      return;
+    }
+    JobId jobID = MRApps.toJobID(jid);
+    Job job = appContext.getJob(jobID);
+    if (job == null) {
+      html.
+        p()._("Sorry, ", jid, " not found.")._();
+      return;
+    }
+    Path confPath = job.getConfFile();
+    try {
+      //Read in the configuration file and put it in a key/value table.
+      FileContext fc = FileContext.getFileContext(confPath.toUri(), conf);
+      Configuration jobConf = new Configuration(false);
+      jobConf.addResource(fc.open(confPath));
+    
+      html.div().h3(confPath.toString())._();
+      TBODY<TABLE<Hamlet>> tbody = html.
+        // Tasks table
+      table("#conf").
+        thead().
+          tr().
+            th(_TH, "key").
+            th(_TH, "value").
+          _().
+        _().
+      tbody();
+      for(Map.Entry<String, String> entry : jobConf) {
+        tbody.
+          tr().
+            td(entry.getKey()).
+            td(entry.getValue()).
+          _();
+      }
+      tbody._().
+      tfoot().
+        tr().
+          th().input("search_init").$type(InputType.text).$name("key").$value("key")._()._().
+          th().input("search_init").$type(InputType.text).$name("value").$value("value")._()._().
+          _().
+        _().
+      _();
+    } catch(IOException e) {
+      LOG.error("Error while reading "+confPath, e);
+      html.p()._("Sorry got an error while reading conf file. ",confPath);
+    }
+  }
+}

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

@@ -22,10 +22,12 @@ import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapreduce.FileSystemCounter;
 import org.apache.hadoop.mapreduce.JobACL;
@@ -50,6 +52,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -258,6 +261,16 @@ public class MockJobs extends MockApps {
       public String getAssignedContainerMgrAddress() {
         return "localhost:9998";
       }
+
+      @Override
+      public long getShuffleFinishTime() {
+        return 0;
+      }
+
+      @Override
+      public long getSortFinishTime() {
+        return 0;
+      }
     };
   }
 
@@ -452,7 +465,7 @@ public class MockJobs extends MockApps {
 
       @Override
       public List<String> getDiagnostics() {
-        throw new UnsupportedOperationException("Not supported yet.");
+        return Collections.<String>emptyList();
       }
 
       @Override
@@ -463,8 +476,18 @@ public class MockJobs extends MockApps {
 
       @Override
       public String getUserName() {
+        return "mock";
+      }
+
+      @Override
+      public Path getConfFile() {
         throw new UnsupportedOperationException("Not supported yet.");
       }
+
+      @Override
+      public Map<JobACL, AccessControlList> getJobACLs() {
+        return Collections.<JobACL, AccessControlList>emptyMap();
+      }
     };
   }
 }

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

@@ -31,6 +31,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -58,6 +59,7 @@ import org.apache.hadoop.mapreduce.v2.app.speculate.Speculator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -461,6 +463,16 @@ public class TestRuntimeEstimators {
     public String getUserName() {
       throw new UnsupportedOperationException("Not supported yet.");
     }
+
+    @Override
+    public Path getConfFile() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public Map<JobACL, AccessControlList> getJobACLs() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
   }
 
   /*
@@ -681,6 +693,16 @@ public class TestRuntimeEstimators {
       throw new UnsupportedOperationException("Not supported yet.");
     }
 
+    @Override
+    public long getShuffleFinishTime() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public long getSortFinishTime() {
+      throw new UnsupportedOperationException("Not supported yet.");
+    }
+
     @Override
     public String getAssignedContainerMgrAddress() {
       throw new UnsupportedOperationException("Not supported yet.");

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

@@ -21,12 +21,17 @@ package org.apache.hadoop.mapreduce.v2.app.webapp;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
 import static org.junit.Assert.assertEquals;
 
+import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -115,18 +120,42 @@ public class TestAMWebApp {
     WebAppTests.testPage(AppView.class, AppContext.class, new TestAppContext());
   }
 
+
+  
   @Test public void testJobView() {
-    WebAppTests.testPage(JobPage.class, AppContext.class, new TestAppContext());
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = getJobParams(appContext);
+    WebAppTests.testPage(JobPage.class, AppContext.class, appContext, params);
   }
 
   @Test public void testTasksView() {
-    WebAppTests.testPage(TasksPage.class, AppContext.class,
-                         new TestAppContext());
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = getTaskParams(appContext);
+    WebAppTests.testPage(TasksPage.class, AppContext.class, appContext, params);
   }
 
   @Test public void testTaskView() {
-    WebAppTests.testPage(TaskPage.class, AppContext.class,
-                         new TestAppContext());
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = getTaskParams(appContext);
+    WebAppTests.testPage(TaskPage.class, AppContext.class, appContext, params);
+  }
+
+  public static Map<String, String> getJobParams(AppContext appContext) {
+    JobId jobId = appContext.getAllJobs().entrySet().iterator().next().getKey();
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(AMParams.JOB_ID, MRApps.toString(jobId));
+    return params;
+  }
+  
+  public static Map<String, String> getTaskParams(AppContext appContext) {
+    JobId jobId = appContext.getAllJobs().entrySet().iterator().next().getKey();
+    Entry<TaskId, Task> e = appContext.getJob(jobId).getTasks().entrySet().iterator().next();
+    e.getValue().getType();
+    Map<String, String> params = new HashMap<String, String>();
+    params.put(AMParams.JOB_ID, MRApps.toString(jobId));
+    params.put(AMParams.TASK_ID, e.getKey().toString());
+    params.put(AMParams.TASK_TYPE, MRApps.taskSymbol(e.getValue().getType()));
+    return params;
   }
 
   public static void main(String[] args) {

+ 14 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/TaskAttemptReport.java

@@ -24,6 +24,10 @@ public interface TaskAttemptReport {
   public abstract float getProgress();
   public abstract long getStartTime();
   public abstract long getFinishTime();
+  /** @return the shuffle finish time. Applicable only for reduce attempts */
+  public abstract long getShuffleFinishTime();
+  /** @return the sort/merge finish time. Applicable only for reduce attempts */
+  public abstract long getSortFinishTime();
   public abstract Counters getCounters();
   public abstract String getDiagnosticInfo();
   public abstract String getStateString();
@@ -39,4 +43,14 @@ public interface TaskAttemptReport {
   public abstract void setStateString(String stateString);
   public abstract void setPhase(Phase phase);
   
+  /** 
+   * Set the shuffle finish time. Applicable only for reduce attempts
+   * @param time the time the shuffle finished.
+   */
+  public abstract void setShuffleFinishTime(long time);
+  /** 
+   * Set the sort/merge finish time. Applicable only for reduce attempts
+   * @param time the time the shuffle finished.
+   */
+  public abstract void setSortFinishTime(long time);
 }

+ 25 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskAttemptReportPBImpl.java

@@ -127,6 +127,31 @@ public class TaskAttemptReportPBImpl extends ProtoBase<TaskAttemptReportProto> i
     maybeInitBuilder();
     builder.setFinishTime((finishTime));
   }
+  
+  @Override
+  public long getShuffleFinishTime() {
+    TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getShuffleFinishTime());
+  }
+
+  @Override
+  public void setShuffleFinishTime(long time) {
+    maybeInitBuilder();
+    builder.setShuffleFinishTime(time);
+  }
+
+  @Override
+  public long getSortFinishTime() {
+    TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
+    return (p.getSortFinishTime());
+  }
+
+  @Override
+  public void setSortFinishTime(long time) {
+    maybeInitBuilder();
+    builder.setSortFinishTime(time);
+  }
+
   @Override
   public TaskAttemptId getTaskAttemptId() {
     TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
@@ -262,7 +287,4 @@ public class TaskAttemptReportPBImpl extends ProtoBase<TaskAttemptReportProto> i
   private Phase convertFromProtoFormat(PhaseProto e) {
     return MRProtoUtils.convertFromProtoFormat(e);
   }
-
-
-
 }  

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/TaskReportPBImpl.java

@@ -132,6 +132,7 @@ public class TaskReportPBImpl extends ProtoBase<TaskReportProto> implements Task
     maybeInitBuilder();
     builder.setStartTime((startTime));
   }
+  
   @Override
   public long getFinishTime() {
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;
@@ -143,6 +144,7 @@ public class TaskReportPBImpl extends ProtoBase<TaskReportProto> implements Task
     maybeInitBuilder();
     builder.setFinishTime((finishTime));
   }
+  
   @Override
   public TaskId getTaskId() {
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;

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

@@ -25,14 +25,20 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.net.URI;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.v2.MRConstants;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@@ -42,12 +48,18 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LocalResourceType;
+import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 
 /**
  * Helper class for MR applications
  */
+@Private
+@Unstable
 public class MRApps extends Apps {
   public static final String JOB = "job";
   public static final String TASK = "task";
@@ -232,4 +244,121 @@ public class MRApps extends Apps {
         jobId.toString() + Path.SEPARATOR + MRConstants.JOB_CONF_FILE);
     return jobFile.toString();
   }
+  
+
+
+  private static long[] parseTimeStamps(String[] strs) {
+    if (null == strs) {
+      return null;
+    }
+    long[] result = new long[strs.length];
+    for(int i=0; i < strs.length; ++i) {
+      result[i] = Long.parseLong(strs[i]);
+    }
+    return result;
+  }
+
+  public static void setupDistributedCache( 
+      Configuration conf, 
+      Map<String, LocalResource> localResources,
+      Map<String, String> env) 
+  throws IOException {
+    
+    // Cache archives
+    parseDistributedCacheArtifacts(conf, localResources, env, 
+        LocalResourceType.ARCHIVE, 
+        DistributedCache.getCacheArchives(conf), 
+        parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)), 
+        getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES), 
+        DistributedCache.getArchiveVisibilities(conf), 
+        DistributedCache.getArchiveClassPaths(conf));
+    
+    // Cache files
+    parseDistributedCacheArtifacts(conf, 
+        localResources, env, 
+        LocalResourceType.FILE, 
+        DistributedCache.getCacheFiles(conf),
+        parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
+        getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
+        DistributedCache.getFileVisibilities(conf),
+        DistributedCache.getFileClassPaths(conf));
+  }
+
+  // TODO - Move this to MR!
+  // Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], 
+  // long[], boolean[], Path[], FileType)
+  private static void parseDistributedCacheArtifacts(
+      Configuration conf,
+      Map<String, LocalResource> localResources,
+      Map<String, String> env,
+      LocalResourceType type,
+      URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], 
+      Path[] pathsToPutOnClasspath) throws IOException {
+
+    if (uris != null) {
+      // Sanity check
+      if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
+          (uris.length != visibilities.length)) {
+        throw new IllegalArgumentException("Invalid specification for " +
+            "distributed-cache artifacts of type " + type + " :" +
+            " #uris=" + uris.length +
+            " #timestamps=" + timestamps.length +
+            " #visibilities=" + visibilities.length
+            );
+      }
+      
+      Map<String, Path> classPaths = new HashMap<String, Path>();
+      if (pathsToPutOnClasspath != null) {
+        for (Path p : pathsToPutOnClasspath) {
+          FileSystem remoteFS = p.getFileSystem(conf);
+          p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
+              remoteFS.getWorkingDirectory()));
+          classPaths.put(p.toUri().getPath().toString(), p);
+        }
+      }
+      for (int i = 0; i < uris.length; ++i) {
+        URI u = uris[i];
+        Path p = new Path(u);
+        FileSystem remoteFS = p.getFileSystem(conf);
+        p = remoteFS.resolvePath(p.makeQualified(remoteFS.getUri(),
+            remoteFS.getWorkingDirectory()));
+        // Add URI fragment or just the filename
+        Path name = new Path((null == u.getFragment())
+          ? p.getName()
+          : u.getFragment());
+        if (name.isAbsolute()) {
+          throw new IllegalArgumentException("Resource name must be relative");
+        }
+        String linkName = name.toUri().getPath();
+        localResources.put(
+            linkName,
+            BuilderUtils.newLocalResource(
+                p.toUri(), type, 
+                visibilities[i]
+                  ? LocalResourceVisibility.PUBLIC
+                  : LocalResourceVisibility.PRIVATE,
+                sizes[i], timestamps[i])
+        );
+        if (classPaths.containsKey(u.getPath())) {
+          MRApps.addToClassPath(env, linkName);
+        }
+      }
+    }
+  }
+  
+  // TODO - Move this to MR!
+  private static long[] getFileSizes(Configuration conf, String key) {
+    String[] strs = conf.getStrings(key);
+    if (strs == null) {
+      return null;
+    }
+    long[] result = new long[strs.length];
+    for(int i=0; i < strs.length; ++i) {
+      result[i] = Long.parseLong(strs[i]);
+    }
+    return result;
+  }
+  
+
+
 }

+ 2 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto

@@ -117,6 +117,8 @@ message TaskAttemptReportProto {
   optional string diagnostic_info = 7;
   optional string state_string = 8;
   optional PhaseProto phase = 9;
+  optional int64 shuffle_finish_time = 10;
+  optional int64 sort_finish_time=11;
 }
 
 enum JobStateProto {

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

@@ -379,46 +379,46 @@ public class JobHistoryParser {
       }
     }
 
-    /** Get the job submit time */
+    /** @return the job submit time */
     public long getSubmitTime() { return submitTime; }
-    /** Get the job finish time */
+    /** @return the job finish time */
     public long getFinishTime() { return finishTime; }
-    /** Get the job id */
+    /** @return the job id */
     public JobID getJobId() { return jobid; }
-    /** Get the user name */
+    /** @return the user name */
     public String getUsername() { return username; }
-    /** Get the job name */
+    /** @return the job name */
     public String getJobname() { return jobname; }
-    /** Get the job queue name */
+    /** @return the job queue name */
     public String getJobQueueName() { return jobQueueName; }
-    /** Get the path for the job configuration file */
+    /** @return the path for the job configuration file */
     public String getJobConfPath() { return jobConfPath; }
-    /** Get the job launch time */
+    /** @return the job launch time */
     public long getLaunchTime() { return launchTime; }
-    /** Get the total number of maps */
+    /** @return the total number of maps */
     public long getTotalMaps() { return totalMaps; }
-    /** Get the total number of reduces */
+    /** @return the total number of reduces */
     public long getTotalReduces() { return totalReduces; }
-    /** Get the total number of failed maps */
+    /** @return the total number of failed maps */
     public long getFailedMaps() { return failedMaps; }
-    /** Get the number of failed reduces */
+    /** @return the number of failed reduces */
     public long getFailedReduces() { return failedReduces; }
-    /** Get the number of finished maps */
+    /** @return the number of finished maps */
     public long getFinishedMaps() { return finishedMaps; }
-    /** Get the number of finished reduces */
+    /** @return the number of finished reduces */
     public long getFinishedReduces() { return finishedReduces; }
-    /** Get the job status */
+    /** @return the job status */
     public String getJobStatus() { return jobStatus; }
     public String getErrorInfo() { return errorInfo; }
-    /** Get the counters for the job */
+    /** @return the counters for the job */
     public Counters getTotalCounters() { return totalCounters; }
-    /** Get the map counters for the job */
+    /** @return the map counters for the job */
     public Counters getMapCounters() { return mapCounters; }
-    /** Get the reduce counters for the job */
+    /** @return the reduce counters for the job */
     public Counters getReduceCounters() { return reduceCounters; }
-    /** Get the map of all tasks in this job */
+    /** @return the map of all tasks in this job */
     public Map<TaskID, TaskInfo> getAllTasks() { return tasksMap; }
-    /** Get the priority of this job */
+    /** @return the priority of this job */
     public String getPriority() { return priority.toString(); }
     public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
   }
@@ -458,27 +458,27 @@ public class JobHistoryParser {
       }
     }
     
-    /** Get the Task ID */
+    /** @return the Task ID */
     public TaskID getTaskId() { return taskId; }
-    /** Get the start time of this task */
+    /** @return the start time of this task */
     public long getStartTime() { return startTime; }
-    /** Get the finish time of this task */
+    /** @return the finish time of this task */
     public long getFinishTime() { return finishTime; }
-    /** Get the task type */
+    /** @return the task type */
     public TaskType getTaskType() { return taskType; }
-    /** Get the split locations */
+    /** @return the split locations */
     public String getSplitLocations() { return splitLocations; }
-    /** Get the counters for this task */
+    /** @return the counters for this task */
     public Counters getCounters() { return counters; }
-    /** Get the task status */
+    /** @return the task status */
     public String getTaskStatus() { return status; }
-    /** Get the attempt Id that caused this task to fail */
+    /** @return the attempt Id that caused this task to fail */
     public TaskAttemptID getFailedDueToAttemptId() {
       return failedDueToAttemptId;
     }
-    /** Get the error */
+    /** @return the error */
     public String getError() { return error; }
-    /** Get the map of all attempts for this task */
+    /** @return the map of all attempts for this task */
     public Map<TaskAttemptID, TaskAttemptInfo> getAllTaskAttempts() {
       return attemptsMap;
     }
@@ -530,33 +530,33 @@ public class JobHistoryParser {
       }
     }
 
-    /** Get the attempt Id */
+    /** @return the attempt Id */
     public TaskAttemptID getAttemptId() { return attemptId; }
-    /** Get the start time of the attempt */
+    /** @return the start time of the attempt */
     public long getStartTime() { return startTime; }
-    /** Get the finish time of the attempt */
+    /** @return the finish time of the attempt */
     public long getFinishTime() { return finishTime; }
-    /** Get the shuffle finish time. Applicable only for reduce attempts */
+    /** @return the shuffle finish time. Applicable only for reduce attempts */
     public long getShuffleFinishTime() { return shuffleFinishTime; }
-    /** Get the sort finish time. Applicable only for reduce attempts */
+    /** @return the sort finish time. Applicable only for reduce attempts */
     public long getSortFinishTime() { return sortFinishTime; }
-    /** Get the map finish time. Applicable only for map attempts */
+    /** @return the map finish time. Applicable only for map attempts */
     public long getMapFinishTime() { return mapFinishTime; }
-    /** Get the error string */
+    /** @return the error string */
     public String getError() { return error; }
-    /** Get the state */
+    /** @return the state */
     public String getState() { return state; }
-    /** Get the task status */
+    /** @return the task status */
     public String getTaskStatus() { return status; }
-    /** Get the task type */
+    /** @return the task type */
     public TaskType getTaskType() { return taskType; }
-    /** Get the tracker name where the attempt executed */
+    /** @return the tracker name where the attempt executed */
     public String getTrackerName() { return trackerName; }
-    /** Get the host name */
+    /** @return the host name */
     public String getHostname() { return hostname; }
-    /** Get the counters for the attempt */
+    /** @return the counters for the attempt */
     public Counters getCounters() { return counters; }
-    /** Get the HTTP port for the tracker */
+    /** @return the HTTP port for the tracker */
     public int getHttpPort() { return httpPort; }
   }
 }

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

@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.mapreduce.jobhistory;
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.TaskStatus;
@@ -28,7 +26,6 @@ import org.apache.hadoop.mapreduce.TaskID;
 import org.apache.hadoop.mapreduce.TaskType;
 
 import org.apache.hadoop.mapred.ProgressSplitsBlock;
-import org.apache.hadoop.mapred.TaskStatus;
 
 import org.apache.avro.util.Utf8;
 

+ 38 - 11
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java

@@ -70,22 +70,26 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   private final Map<TaskId, Task> mapTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
   private final String user;
+  private final Path confFile;
   
   private List<TaskAttemptCompletionEvent> completionEvents = null;
   private JobInfo jobInfo;
 
   public CompletedJob(Configuration conf, JobId jobId, Path historyFile, 
-      boolean loadTasks, String userName) throws IOException {
+      boolean loadTasks, String userName, Path confFile) throws IOException {
     LOG.info("Loading job: " + jobId + " from file: " + historyFile);
     this.conf = conf;
     this.jobId = jobId;
+    this.confFile = confFile;
     
     loadFullHistoryData(loadTasks, historyFile);
 
     user = userName;
     counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
     diagnostics.add(jobInfo.getErrorInfo());
-    report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class);
+    report =
+        RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
+            JobReport.class);
     report.setJobId(jobId);
     report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
     report.setStartTime(jobInfo.getLaunchTime());
@@ -192,11 +196,12 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
 
       int attemptRunTime = -1;
       if (taskAttempt.getLaunchTime() != 0 && taskAttempt.getFinishTime() != 0) {
-        attemptRunTime = (int) (taskAttempt.getFinishTime() - taskAttempt
-            .getLaunchTime());
+        attemptRunTime =
+            (int) (taskAttempt.getFinishTime() - taskAttempt.getLaunchTime());
       }
       // Default to KILLED
-      TaskAttemptCompletionEventStatus taceStatus = TaskAttemptCompletionEventStatus.KILLED;
+      TaskAttemptCompletionEventStatus taceStatus =
+          TaskAttemptCompletionEventStatus.KILLED;
       String taStateString = taskAttempt.getState().toString();
       try {
         taceStatus = TaskAttemptCompletionEventStatus.valueOf(taStateString);
@@ -222,7 +227,8 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   }
 
   //History data is leisurely loaded when task level data is requested
-  private synchronized void loadFullHistoryData(boolean loadTasks, Path historyFileAbsolute) throws IOException {
+  private synchronized void loadFullHistoryData(boolean loadTasks,
+      Path historyFileAbsolute) throws IOException {
     LOG.info("Loading history file: [" + historyFileAbsolute + "]");
     if (jobInfo != null) {
       return; //data already loaded
@@ -230,11 +236,13 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     
     if (historyFileAbsolute != null) {
       try {
-      JobHistoryParser parser = new JobHistoryParser(historyFileAbsolute.getFileSystem(conf), historyFileAbsolute);
-      jobInfo = parser.parse();
+        JobHistoryParser parser =
+            new JobHistoryParser(historyFileAbsolute.getFileSystem(conf),
+                historyFileAbsolute);
+        jobInfo = parser.parse();
       } catch (IOException e) {
-        throw new YarnException("Could not load history file " + historyFileAbsolute,
-            e);
+        throw new YarnException("Could not load history file "
+            + historyFileAbsolute, e);
       }
     } else {
       throw new IOException("History file not found");
@@ -293,7 +301,8 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   }
 
   @Override
-  public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
+  public
+      boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
     if (!UserGroupInformation.isSecurityEnabled()) {
       return true;
     }
@@ -304,8 +313,26 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
         jobInfo.getUsername(), jobACL);
   }
   
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getJobACLs()
+   */
+  @Override
+  public  Map<JobACL, AccessControlList> getJobACLs() {
+    return jobInfo.getJobACLs();
+  }
+  
   @Override
   public String getUserName() {
     return user;
   }
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
+   */
+  @Override
+  public Path getConfFile() {
+    return confFile;
+  }
 }

+ 12 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedTaskAttempt.java

@@ -71,6 +71,8 @@ public class CompletedTaskAttempt implements TaskAttempt {
     report.setStartTime(attemptInfo.getStartTime());
     
     report.setFinishTime(attemptInfo.getFinishTime());
+    report.setShuffleFinishTime(attemptInfo.getShuffleFinishTime());
+    report.setSortFinishTime(attemptInfo.getSortFinishTime());
     if (localDiagMessage != null) {
       report.setDiagnosticInfo(attemptInfo.getError() + ", " + localDiagMessage);
     } else {
@@ -158,10 +160,19 @@ public class CompletedTaskAttempt implements TaskAttempt {
   public long getFinishTime() {
     return report.getFinishTime();
   }
+  
+  @Override
+  public long getShuffleFinishTime() {
+    return report.getShuffleFinishTime();
+  }
+
+  @Override
+  public long getSortFinishTime() {
+    return report.getSortFinishTime();
+  }
 
   @Override
   public int getShufflePort() {
     throw new UnsupportedOperationException("Not supported yet.");
   }
-
 }

+ 118 - 195
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce.v2.hs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
@@ -36,8 +35,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -87,18 +84,18 @@ public class JobHistory extends AbstractService implements HistoryContext   {
 
   private static final Log SUMMARY_LOG = LogFactory.getLog(JobSummary.class);
 
-  private static final Pattern DATE_PATTERN = Pattern
-      .compile("([0-1]?[0-9])/([0-3]?[0-9])/((?:2[0-9])[0-9][0-9])");
-
   /*
    * TODO Get rid of this once JobId has it's own comparator
    */
-  private static final Comparator<JobId> JOB_ID_COMPARATOR = new Comparator<JobId>() {
+  private static final Comparator<JobId> JOB_ID_COMPARATOR = 
+    new Comparator<JobId>() {
     @Override
     public int compare(JobId o1, JobId o2) {
-      if (o1.getAppId().getClusterTimestamp() > o2.getAppId().getClusterTimestamp()) {
+      if (o1.getAppId().getClusterTimestamp() > 
+          o2.getAppId().getClusterTimestamp()) {
         return 1;
-      } else if (o1.getAppId().getClusterTimestamp() < o2.getAppId().getClusterTimestamp()) {
+      } else if (o1.getAppId().getClusterTimestamp() < 
+          o2.getAppId().getClusterTimestamp()) {
         return -1;
       } else {
         return o1.getId() - o2.getId();
@@ -106,7 +103,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     }
   };
   
-  private static String DONE_BEFORE_SERIAL_TAIL = JobHistoryUtils.doneSubdirsBeforeSerialTail();
+  private static String DONE_BEFORE_SERIAL_TAIL = 
+    JobHistoryUtils.doneSubdirsBeforeSerialTail();
   
   /**
    * Maps between a serial number (generated based on jobId) and the timestamp
@@ -114,29 +112,32 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * Facilitates jobId based searches.
    * If a jobId is not found in this list - it will not be found.
    */
-  private final SortedMap<String, Set<String>> idToDateString = new ConcurrentSkipListMap<String, Set<String>>();
+  private final SortedMap<String, Set<String>> idToDateString = 
+    new ConcurrentSkipListMap<String, Set<String>>();
 
   //Maintains minimal details for recent jobs (parsed from history file name).
   //Sorted on Job Completion Time.
-  private final SortedMap<JobId, MetaInfo> jobListCache = new ConcurrentSkipListMap<JobId, MetaInfo>(
-      JOB_ID_COMPARATOR);
+  private final SortedMap<JobId, MetaInfo> jobListCache = 
+    new ConcurrentSkipListMap<JobId, MetaInfo>(JOB_ID_COMPARATOR);
   
   
   // Re-use exisiting MetaInfo objects if they exist for the specific JobId. (synchronization on MetaInfo)
   // Check for existance of the object when using iterators.
-  private final SortedMap<JobId, MetaInfo> intermediateListCache = new ConcurrentSkipListMap<JobId, JobHistory.MetaInfo>(
-      JOB_ID_COMPARATOR);
+  private final SortedMap<JobId, MetaInfo> intermediateListCache = 
+    new ConcurrentSkipListMap<JobId, JobHistory.MetaInfo>(JOB_ID_COMPARATOR);
   
   //Maintains a list of known done subdirectories. Not currently used.
   private final Set<Path> existingDoneSubdirs = new HashSet<Path>();
   
-  private final SortedMap<JobId, Job> loadedJobCache = new ConcurrentSkipListMap<JobId, Job>(
-      JOB_ID_COMPARATOR);
+  private final SortedMap<JobId, Job> loadedJobCache = 
+    new ConcurrentSkipListMap<JobId, Job>(JOB_ID_COMPARATOR);
 
   /**
-   * Maintains a mapping between intermediate user directories and the last known modification time.
+   * Maintains a mapping between intermediate user directories and the last 
+   * known modification time.
    */
-  private Map<String, Long> userDirModificationTimeMap = new HashMap<String, Long>();
+  private Map<String, Long> userDirModificationTimeMap = 
+    new HashMap<String, Long>();
   
   //The number of jobs to maintain in the job list cache.
   private int jobListCacheSize;
@@ -187,7 +188,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
     serialNumberLowDigits = debugMode ? 1 : 3;
     serialNumberFormat = ("%0"
-        + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits) + "d");
+        + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS 
+            + serialNumberLowDigits) + "d");
 
     String doneDirPrefix = null;
     doneDirPrefix = JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
@@ -195,9 +197,11 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
           new Path(doneDirPrefix));
       doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
-      mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
+      mkdir(doneDirFc, doneDirPrefixPath, new FsPermission(
+          JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION));
     } catch (IOException e) {
-      throw new YarnException("Error creating done directory: [" + doneDirPrefixPath + "]", e);
+      throw new YarnException("Error creating done directory: [" +
+          doneDirPrefixPath + "]", e);
     }
 
     String intermediateDoneDirPrefix = null;
@@ -208,21 +212,27 @@ public class JobHistory extends AbstractService implements HistoryContext   {
           .makeQualified(new Path(intermediateDoneDirPrefix));
       intermediateDoneDirFc = FileContext.getFileContext(
           intermediateDoneDirPath.toUri(), conf);
-      mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
+      mkdir(intermediateDoneDirFc, intermediateDoneDirPath, new FsPermission(
+          JobHistoryUtils.HISTORY_INTERMEDIATE_DONE_DIR_PERMISSIONS.toShort()));
     } catch (IOException e) {
       LOG.info("error creating done directory on dfs " + e);
-      throw new YarnException("Error creating intermediate done directory: [" + intermediateDoneDirPath + "]", e);
+      throw new YarnException("Error creating intermediate done directory: [" 
+          + intermediateDoneDirPath + "]", e);
     }
     
     
     
-    jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE, DEFAULT_JOBLIST_CACHE_SIZE);
-    loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE, DEFAULT_LOADEDJOB_CACHE_SIZE);
-    dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE, DEFAULT_DATESTRING_CACHE_SIZE);
+    jobListCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_JOBLIST_CACHE_SIZE,
+        DEFAULT_JOBLIST_CACHE_SIZE);
+    loadedJobCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_LOADED_JOB_CACHE_SIZE,
+        DEFAULT_LOADEDJOB_CACHE_SIZE);
+    dateStringCacheSize = conf.getInt(JHAdminConfig.MR_HISTORY_DATESTRING_CACHE_SIZE,
+        DEFAULT_DATESTRING_CACHE_SIZE);
     moveThreadInterval =
         conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS,
             DEFAULT_MOVE_THREAD_INTERVAL);
-    numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT, DEFAULT_MOVE_THREAD_COUNT);
+    numMoveThreads = conf.getInt(JHAdminConfig.MR_HISTORY_MOVE_THREAD_COUNT,
+        DEFAULT_MOVE_THREAD_COUNT);
     try {
       initExisting();
     } catch (IOException e) {
@@ -254,19 +264,21 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   @Override
   public void start() {
     //Start moveIntermediatToDoneThread
-    moveIntermediateToDoneRunnable = new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
+    moveIntermediateToDoneRunnable = 
+      new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
     moveIntermediateToDoneThread = new Thread(moveIntermediateToDoneRunnable);
     moveIntermediateToDoneThread.setName("MoveIntermediateToDoneScanner");
     moveIntermediateToDoneThread.start();
     
     //Start historyCleaner
-    boolean startCleanerService = conf.getBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
+    boolean startCleanerService = conf.getBoolean(
+        JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
     if (startCleanerService) {
-      long maxAgeOfHistoryFiles = conf.getLong(JHAdminConfig.MR_HISTORY_MAX_AGE_MS,
-          DEFAULT_HISTORY_MAX_AGE);
+      long maxAgeOfHistoryFiles = conf.getLong(
+          JHAdminConfig.MR_HISTORY_MAX_AGE_MS, DEFAULT_HISTORY_MAX_AGE);
     cleanerScheduledExecutor = new ScheduledThreadPoolExecutor(1);
-      long runInterval = conf.getLong(JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS,
-          DEFAULT_RUN_INTERVAL);
+      long runInterval = conf.getLong(
+          JHAdminConfig.MR_HISTORY_CLEANER_INTERVAL_MS, DEFAULT_RUN_INTERVAL);
       cleanerScheduledExecutor
           .scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles),
               30 * 1000l, runInterval, TimeUnit.MILLISECONDS);
@@ -331,13 +343,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   
   private void removeDirectoryFromSerialNumberIndex(Path serialDirPath) {
     String serialPart = serialDirPath.getName();
-    String timeStampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
+    String timeStampPart = 
+      JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
     if (timeStampPart == null) {
-      LOG.warn("Could not find timestamp portion from path: " + serialDirPath.toString() +". Continuing with next");
+      LOG.warn("Could not find timestamp portion from path: " + 
+          serialDirPath.toString() +". Continuing with next");
       return;
     }
     if (serialPart == null) {
-      LOG.warn("Could not find serial portion from path: " + serialDirPath.toString() + ". Continuing with next");
+      LOG.warn("Could not find serial portion from path: " + 
+          serialDirPath.toString() + ". Continuing with next");
       return;
     }
     if (idToDateString.containsKey(serialPart)) {
@@ -355,13 +370,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       LOG.debug("Adding "+serialDirPath+" to serial index");
     }
     String serialPart = serialDirPath.getName();
-    String timestampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
+    String timestampPart = 
+      JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
     if (timestampPart == null) {
-      LOG.warn("Could not find timestamp portion from path: " + serialDirPath.toString() +". Continuing with next");
+      LOG.warn("Could not find timestamp portion from path: " + 
+          serialDirPath.toString() +". Continuing with next");
       return;
     }
     if (serialPart == null) {
-      LOG.warn("Could not find serial portion from path: " + serialDirPath.toString() + ". Continuing with next");
+      LOG.warn("Could not find serial portion from path: " + 
+          serialDirPath.toString() + ". Continuing with next");
     }
     addToSerialNumberIndex(serialPart, timestampPart);
   }
@@ -400,7 +418,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     }
   }
   
-  private static List<FileStatus> scanDirectory(Path path, FileContext fc, PathFilter pathFilter) throws IOException {
+  private static List<FileStatus> scanDirectory(Path path, FileContext fc,
+      PathFilter pathFilter) throws IOException {
     path = fc.makeQualified(path);
     List<FileStatus> jhStatusList = new ArrayList<FileStatus>();
       RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
@@ -414,7 +433,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     return jhStatusList;
   }
   
-  private static List<FileStatus> scanDirectoryForHistoryFiles(Path path, FileContext fc) throws IOException {
+  private static List<FileStatus> scanDirectoryForHistoryFiles(Path path,
+      FileContext fc) throws IOException {
     return scanDirectory(path, fc, JobHistoryUtils.getHistoryFileFilter());
   }
   
@@ -425,7 +445,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * @return
    */
   private List<FileStatus> findTimestampedDirectories() throws IOException {
-    List<FileStatus> fsList = JobHistoryUtils.localGlobber(doneDirFc, doneDirPrefixPath, DONE_BEFORE_SERIAL_TAIL);
+    List<FileStatus> fsList = JobHistoryUtils.localGlobber(doneDirFc, 
+        doneDirPrefixPath, DONE_BEFORE_SERIAL_TAIL);
     return fsList;
   }
     
@@ -434,7 +455,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    */
   private void addToJobListCache(JobId jobId, MetaInfo metaInfo) {
     if(LOG.isDebugEnabled()) {
-      LOG.debug("Adding "+jobId+" to job list cache with "+metaInfo.getJobIndexInfo());
+      LOG.debug("Adding "+jobId+" to job list cache with "
+          +metaInfo.getJobIndexInfo());
     }
     jobListCache.put(jobId, metaInfo);
     if (jobListCache.size() > jobListCacheSize) {
@@ -462,14 +484,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * @throws IOException
    */
   private void scanIntermediateDirectory() throws IOException {
-    List<FileStatus> userDirList = JobHistoryUtils.localGlobber(intermediateDoneDirFc, intermediateDoneDirPath, "");
+    List<FileStatus> userDirList = 
+      JobHistoryUtils.localGlobber(intermediateDoneDirFc, intermediateDoneDirPath, "");
     
     for (FileStatus userDir : userDirList) {
       String name = userDir.getPath().getName();
       long newModificationTime = userDir.getModificationTime();
       boolean shouldScan = false;
       synchronized (userDirModificationTimeMap) {
-        if (!userDirModificationTimeMap.containsKey(name) || newModificationTime > userDirModificationTimeMap.get(name)) {
+        if (!userDirModificationTimeMap.containsKey(name) || newModificationTime
+            > userDirModificationTimeMap.get(name)) {
             shouldScan = true;
             userDirModificationTimeMap.put(name, newModificationTime);
         }  
@@ -514,9 +538,11 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * @return A MetaInfo object for the jobId, null if not found.
    * @throws IOException
    */
-  private MetaInfo getJobMetaInfo(List<FileStatus> fileStatusList, JobId jobId) throws IOException {
+  private MetaInfo getJobMetaInfo(List<FileStatus> fileStatusList, JobId jobId) 
+  throws IOException {
     for (FileStatus fs : fileStatusList) {
-      JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
+      JobIndexInfo jobIndexInfo = 
+        FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
       if (jobIndexInfo.getJobId().equals(jobId)) {
         String confFileName = JobHistoryUtils
             .getIntermediateConfFileName(jobIndexInfo.getJobId());
@@ -549,7 +575,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     }
     for (String timestampPart : dateStringSet) {
       Path logDir = canonicalHistoryLogPath(jobId, timestampPart);
-      List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir, doneDirFc);
+      List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir,
+          doneDirFc);
       MetaInfo metaInfo = getJobMetaInfo(fileStatusList, jobId);
       if (metaInfo != null) {
         return metaInfo;
@@ -559,7 +586,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   }
   
   /**
-   * Checks for the existence of the job history file in the interemediate directory.
+   * Checks for the existence of the job history file in the intermediate 
+   * directory.
    * @param jobId
    * @return
    * @throws IOException
@@ -586,7 +614,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     
     MoveIntermediateToDoneRunnable(long sleepTime, int numMoveThreads) {
       this.sleepTime = sleepTime;
-      moveToDoneExecutor = new ThreadPoolExecutor(1, numMoveThreads, 1, TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
+      moveToDoneExecutor = new ThreadPoolExecutor(1, numMoveThreads, 1, 
+          TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>());
       running = true;
     }
   
@@ -604,7 +633,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
                 try {
                 moveToDone(metaInfo);
                 } catch (IOException e) {
-                  LOG.info("Failed to process metaInfo for job: " + metaInfo.jobIndexInfo.getJobId(), e);
+                  LOG.info("Failed to process metaInfo for job: " + 
+                      metaInfo.jobIndexInfo.getJobId(), e);
                 }
               }
             });
@@ -629,38 +659,17 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     synchronized(metaInfo) {
       try {
         Job job = new CompletedJob(conf, metaInfo.getJobIndexInfo().getJobId(), 
-            metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser());
+            metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser(),
+            metaInfo.getConfFile());
         addToLoadedJobCache(job);
         return job;
       } catch (IOException e) {
-        throw new YarnException("Could not find/load job: " + metaInfo.getJobIndexInfo().getJobId(), e);
+        throw new YarnException("Could not find/load job: " + 
+            metaInfo.getJobIndexInfo().getJobId(), e);
       }
     }
   }
   
-  private SortedMap<JobId, JobIndexInfo> getAllJobsMetaInfo() {
-    SortedMap<JobId, JobIndexInfo> result = new TreeMap<JobId, JobIndexInfo>(JOB_ID_COMPARATOR);
-      try {
-      scanIntermediateDirectory();
-      } catch (IOException e) {
-      LOG.warn("Failed to scan intermediate directory", e);
-        throw new YarnException(e);
-      }
-    for (JobId jobId : intermediateListCache.keySet()) {
-      MetaInfo mi = intermediateListCache.get(jobId);
-      if (mi != null) {
-        result.put(jobId, mi.getJobIndexInfo());
-      }
-    }
-    for (JobId jobId : jobListCache.keySet()) {
-      MetaInfo mi = jobListCache.get(jobId);
-      if (mi != null) {
-        result.put(jobId, mi.getJobIndexInfo());
-      }
-    }
-    return result;
-  }
-  
   private Map<JobId, Job> getAllJobsInternal() {
     //TODO This should ideally be using getAllJobsMetaInfo
     // or get rid of that method once Job has APIs for user, finishTime etc.
@@ -746,108 +755,6 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     return null;
   }
   
-  /**
-   * Searches cached jobs for the specified criteria (AND). Ignores the criteria if null.
-   * @param soughtUser
-   * @param soughtJobNameSubstring
-   * @param soughtDateStrings
-   * @return
-   */
-  private Map<JobId, Job> findJobs(String soughtUser, String soughtJobNameSubstring, String[] soughtDateStrings) {
-    boolean searchUser = true;
-    boolean searchJobName = true;
-    boolean searchDates = true;
-    List<Calendar> soughtCalendars = null;
-    
-    if (soughtUser == null) {
-      searchUser = false;
-    }
-    if (soughtJobNameSubstring == null) {
-      searchJobName = false; 
-    }
-    if (soughtDateStrings == null) {
-      searchDates = false;
-    } else {
-      soughtCalendars = getSoughtDateAsCalendar(soughtDateStrings);
-    }
-    
-    Map<JobId, Job> resultMap = new TreeMap<JobId, Job>();
-    
-    SortedMap<JobId, JobIndexInfo> allJobs = getAllJobsMetaInfo();
-    for (Map.Entry<JobId, JobIndexInfo> entry : allJobs.entrySet()) {
-      JobId jobId = entry.getKey();
-      JobIndexInfo indexInfo = entry.getValue();
-      String jobName = indexInfo.getJobName();
-      String jobUser = indexInfo.getUser();
-      long finishTime = indexInfo.getFinishTime();
-    
-      if (searchUser) {
-        if (!soughtUser.equals(jobUser)) {
-          continue;
-        }
-      }
-      
-      if (searchJobName) {
-        if (!jobName.contains(soughtJobNameSubstring)) {
-          continue;
-        }
-      }
-      
-      if (searchDates) {
-        boolean matchedDate = false;
-        Calendar jobCal = Calendar.getInstance();
-        jobCal.setTimeInMillis(finishTime);
-        for (Calendar cal : soughtCalendars) {
-          if (jobCal.get(Calendar.YEAR) == cal.get(Calendar.YEAR) &&
-              jobCal.get(Calendar.MONTH) == cal.get(Calendar.MONTH) &&
-              jobCal.get(Calendar.DAY_OF_MONTH) == cal.get(Calendar.DAY_OF_MONTH)) {
-            matchedDate = true;
-            break;
-          }
-        }
-        if (!matchedDate) {
-          break;
-        }
-      }
-      resultMap.put(jobId, new PartialJob(indexInfo, jobId));
-    }
-    return resultMap;
-  }
-  
-  private List<Calendar> getSoughtDateAsCalendar(String [] soughtDateStrings) {
-    List<Calendar> soughtCalendars = new ArrayList<Calendar>();
-    for (int i = 0 ; i < soughtDateStrings.length ; i++) {
-      String soughtDate = soughtDateStrings[i];
-      if (soughtDate.length() != 0) {
-        Matcher m = DATE_PATTERN.matcher(soughtDate);
-        if (m.matches()) {
-          String yyyyPart = m.group(3);
-          String mmPart = m.group(1);
-          String ddPart = m.group(2);
-          
-          if (yyyyPart.length() == 2) {
-            yyyyPart = "20" + yyyyPart;
-          }
-          if (mmPart.length() == 1) {
-            mmPart = "0" + mmPart;
-          }
-          if (ddPart.length() == 1) {
-            ddPart = "0" + ddPart;
-          }
-          Calendar soughtCal = Calendar.getInstance();
-          soughtCal.set(Calendar.YEAR, Integer.parseInt(yyyyPart));
-          soughtCal.set(Calendar.MONTH, Integer.parseInt(mmPart) - 1);
-          soughtCal.set(Calendar.DAY_OF_MONTH, Integer.parseInt(ddPart) -1);
-          soughtCalendars.add(soughtCal);
-        }
-      }
-    }
-    return soughtCalendars;
-  }
-  
-  
-
-  
   private void moveToDone(MetaInfo metaInfo) throws IOException {
     long completeTime = metaInfo.getJobIndexInfo().getFinishTime();
     if (completeTime == 0) completeTime = System.currentTimeMillis();
@@ -890,26 +797,31 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     try {
       maybeMakeSubdirectory(targetDir);
     } catch (IOException e) {
-      LOG.warn("Failed creating subdirectory: " + targetDir + " while attempting to move files for jobId: " + jobId);
+      LOG.warn("Failed creating subdirectory: " + targetDir + 
+          " while attempting to move files for jobId: " + jobId);
       throw e;
     }
     synchronized (metaInfo) {
       if (historyFile != null) {
-        Path toPath = doneDirFc.makeQualified(new Path(targetDir, historyFile.getName()));
+        Path toPath = doneDirFc.makeQualified(new Path(targetDir, 
+            historyFile.getName()));
         try {
           moveToDoneNow(historyFile, toPath);
         } catch (IOException e) {
-          LOG.warn("Failed to move file: " + historyFile + " for jobId: " + jobId);
+          LOG.warn("Failed to move file: " + historyFile + " for jobId: "
+              + jobId);
           throw e;
         }
         metaInfo.setHistoryFile(toPath);
       }
       if (confFile != null) {
-        Path toPath = doneDirFc.makeQualified(new Path(targetDir, confFile.getName()));
+        Path toPath = doneDirFc.makeQualified(new Path(targetDir, 
+            confFile.getName()));
         try {
           moveToDoneNow(confFile, toPath);
         } catch (IOException e) {
-          LOG.warn("Failed to move file: " + historyFile + " for jobId: " + jobId);
+          LOG.warn("Failed to move file: " + historyFile + " for jobId: " 
+              + jobId);
           throw e;
         }
         metaInfo.setConfFile(toPath);
@@ -953,7 +865,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       }
     } catch (FileNotFoundException fnfE) {
       try {
-        FsPermission fsp = new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
+        FsPermission fsp = 
+          new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
         doneDirFc.mkdir(path, fsp, true);
         FileStatus fsStatus = doneDirFc.getFileStatus(path);
         LOG.info("Perms after creating " + fsStatus.getPermission().toShort()
@@ -972,12 +885,15 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   }
   
   private Path canonicalHistoryLogPath(JobId id, String timestampComponent) {
-    return new Path(doneDirPrefixPath, JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
+    return new Path(doneDirPrefixPath, 
+        JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
   }
   
   private Path canonicalHistoryLogPath(JobId id, long millisecondTime) {
-    String timestampComponent = JobHistoryUtils.timestampDirectoryComponent(millisecondTime, debugMode);
-    return new Path(doneDirPrefixPath, JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
+    String timestampComponent = 
+      JobHistoryUtils.timestampDirectoryComponent(millisecondTime, debugMode);
+    return new Path(doneDirPrefixPath, 
+        JobHistoryUtils.historyLogSubdirectory(id, timestampComponent, serialNumberFormat));
   }  
   
 
@@ -1033,12 +949,13 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     private Path summaryFile;
     JobIndexInfo jobIndexInfo;
 
-    MetaInfo(Path historyFile, Path confFile, Path summaryFile, JobIndexInfo jobIndexInfo) {
+    MetaInfo(Path historyFile, Path confFile, Path summaryFile, 
+        JobIndexInfo jobIndexInfo) {
       this.historyFile = historyFile;
       this.confFile = confFile;
       this.summaryFile = summaryFile;
       this.jobIndexInfo = jobIndexInfo;
-      }
+    }
 
     Path getHistoryFile() { return historyFile; }
     Path getConfFile() { return confFile; }
@@ -1073,13 +990,19 @@ public class JobHistory extends AbstractService implements HistoryContext   {
         //Sort in ascending order. Relies on YYYY/MM/DD/Serial
         Collections.sort(serialDirList);
         for (FileStatus serialDir : serialDirList) {
-          List<FileStatus> historyFileList = scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
+          List<FileStatus> historyFileList = 
+            scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
           for (FileStatus historyFile : historyFileList) {
-            JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(historyFile.getPath().getName());
-            long effectiveTimestamp = getEffectiveTimestamp(jobIndexInfo.getFinishTime(), historyFile);
+            JobIndexInfo jobIndexInfo = 
+              FileNameIndexUtils.getIndexInfo(historyFile.getPath().getName());
+            long effectiveTimestamp = 
+              getEffectiveTimestamp(jobIndexInfo.getFinishTime(), historyFile);
             if (shouldDelete(effectiveTimestamp)) {
-              String confFileName = JobHistoryUtils.getIntermediateConfFileName(jobIndexInfo.getJobId());
-              MetaInfo metaInfo = new MetaInfo(historyFile.getPath(), new Path(historyFile.getPath().getParent(), confFileName), null, jobIndexInfo);
+              String confFileName = 
+                JobHistoryUtils.getIntermediateConfFileName(jobIndexInfo.getJobId());
+              MetaInfo metaInfo = new MetaInfo(historyFile.getPath(),
+                  new Path(historyFile.getPath().getParent(), confFileName), 
+                  null, jobIndexInfo);
               delete(metaInfo);
             } else {
               halted = true;

+ 7 - 6
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java

@@ -74,14 +74,15 @@ public class JobHistoryServer extends CompositeService {
 
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
-    JobHistoryServer server = null;
     try {
-      server = new JobHistoryServer();
+      JobHistoryServer jobHistoryServer = new JobHistoryServer();
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(jobHistoryServer));
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
-      server.init(conf);
-      server.start();
-    } catch (Throwable e) {
-      LOG.fatal(StringUtils.stringifyException(e));
+      jobHistoryServer.init(conf);
+      jobHistoryServer.start();
+    } catch (Throwable t) {
+      LOG.fatal("Error starting JobHistoryServer", t);
       System.exit(-1);
     }
   }

+ 12 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/PartialJob.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.hs;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
@@ -32,6 +33,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 import clover.org.apache.log4j.Logger;
@@ -147,4 +149,14 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
     return jobIndexInfo.getUser();
   }
 
+  @Override
+  public Path getConfFile() {
+    throw new IllegalStateException("Not implemented yet");
+  }
+
+  @Override
+  public Map<JobACL, AccessControlList> getJobACLs() {
+    throw new IllegalStateException("Not implemented yet");
+  }
+
 }

+ 97 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAttemptsPage.java

@@ -0,0 +1,97 @@
+/**
+* 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.hs.webapp;
+
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.ATTEMPT_STATE;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+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.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.yarn.webapp.SubView;
+
+import com.google.inject.Inject;
+
+/**
+ * Render a page showing the attempts made of a given type and a given job.
+ */
+public class HsAttemptsPage extends HsTaskPage {
+  static class FewAttemptsBlock extends HsTaskPage.AttemptsBlock {
+    @Inject
+    FewAttemptsBlock(App ctx) {
+      super(ctx);
+    }
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock#isValidRequest()
+     * Verify that a job is given.
+     */
+    @Override
+    protected boolean isValidRequest() {
+      return app.getJob() != null;
+    }
+
+    /*
+     * (non-Javadoc)
+     * @see org.apache.hadoop.mapreduce.v2.hs.webapp.HsTaskPage.AttemptsBlock#getTaskAttempts()
+     * @return the attempts that are for a given job and a specific type/state.
+     */
+    @Override
+    protected Collection<TaskAttempt> getTaskAttempts() {
+      List<TaskAttempt> fewTaskAttemps = new ArrayList<TaskAttempt>();
+      String taskTypeStr = $(TASK_TYPE);
+      TaskType taskType = MRApps.taskType(taskTypeStr);
+      String attemptStateStr = $(ATTEMPT_STATE);
+      TaskAttemptStateUI neededState = MRApps
+          .taskAttemptState(attemptStateStr);
+      Job j = app.getJob();
+      Map<TaskId, Task> tasks = j.getTasks(taskType);
+      for (Task task : tasks.values()) {
+        Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
+        for (TaskAttempt attempt : attempts.values()) {
+          if (neededState.correspondsTo(attempt.getState())) {
+            fewTaskAttemps.add(attempt);
+          }
+        }
+      }
+      return fewTaskAttemps;
+    }
+  }
+
+  /**
+   * The content will render a different set of task attempts.
+   * @return FewAttemptsBlock.class
+   */
+  @Override
+  protected Class<? extends SubView> content() {
+    return FewAttemptsBlock.class;
+  }
+}

+ 99 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsConfPage.java

@@ -0,0 +1,99 @@
+/**
+* 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.hs.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 give job in it.
+ */
+public class HsConfPage extends HsView {
+
+  /*
+   * (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");
+
+    //Override the default nav config
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
+  }
+
+  /**
+   * The body of this block is the configuration block.
+   * @return HsConfBlock.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";
+  }
+}

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

@@ -78,7 +78,16 @@ public class HsController extends AppController {
   protected Class<? extends View> taskPage() {
     return HsTaskPage.class;
   }
-
+  
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.webapp.AppController#attemptsPage()
+   */
+  @Override
+  protected Class<? extends View> attemptsPage() {
+    return HsAttemptsPage.class;
+  }
+  
   // Need all of these methods here also as Guice doesn't look into parent
   // classes.
   
@@ -127,6 +136,21 @@ public class HsController extends AppController {
     super.attempts();
   }
   
+  /**
+   * @return the page that will be used to render the /conf page
+   */
+  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.
    */

+ 69 - 77
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java

@@ -20,11 +20,14 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
 import com.google.inject.Inject;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
@@ -32,12 +35,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
-import static org.apache.hadoop.yarn.util.StringHelper.*;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 
 /**
@@ -46,21 +50,18 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 public class HsJobBlock extends HtmlBlock {
   final AppContext appContext;
 
-  int runningMapTasks = 0;
-  int pendingMapTasks = 0;
-  int runningReduceTasks = 0;
-  int pendingReduceTasks = 0;
-
-  int newMapAttempts = 0;
-  int runningMapAttempts = 0;
   int killedMapAttempts = 0;
   int failedMapAttempts = 0;
   int successfulMapAttempts = 0;
-  int newReduceAttempts = 0;
-  int runningReduceAttempts = 0;
   int killedReduceAttempts = 0;
   int failedReduceAttempts = 0;
   int successfulReduceAttempts = 0;
+  long avgMapTime = 0;
+  long avgReduceTime = 0;
+  long avgShuffleTime = 0;
+  long avgSortTime = 0;
+  int numMaps;
+  int numReduces;
 
   @Inject HsJobBlock(AppContext appctx) {
     appContext = appctx;
@@ -84,9 +85,9 @@ public class HsJobBlock extends HtmlBlock {
         p()._("Sorry, ", jid, " not found.")._();
       return;
     }
+    Map<JobACL, AccessControlList> acls = job.getJobACLs();
+    
     JobReport jobReport = job.getReport();
-    String mapPct = percent(jobReport.getMapProgress());
-    String reducePct = percent(jobReport.getReduceProgress());
     int mapTasks = job.getTotalMaps();
     int mapTasksComplete = job.getCompletedMaps();
     int reduceTasks = job.getTotalReduces();
@@ -94,13 +95,38 @@ public class HsJobBlock extends HtmlBlock {
     long startTime = jobReport.getStartTime();
     long finishTime = jobReport.getFinishTime();
     countTasksAndAttempts(job);
-    info("Job Overview").
+    ResponseInfo infoBlock = info("Job Overview").
         _("Job Name:", job.getName()).
+        _("User Name:", job.getUserName()).
         _("State:", job.getState()).
         _("Uberized:", job.isUber()).
         _("Started:", new Date(startTime)).
+        _("Finished:", new Date(finishTime)).
         _("Elapsed:", StringUtils.formatTime(
-            Times.elapsed(startTime, finishTime)));
+            Times.elapsed(startTime, finishTime, false)));
+    
+    List<String> diagnostics = job.getDiagnostics();
+    if(diagnostics != null && !diagnostics.isEmpty()) {
+      StringBuffer b = new StringBuffer();
+      for(String diag: diagnostics) {
+        b.append(diag);
+      }
+      infoBlock._("Diagnostics:", b.toString());
+    }
+
+    if(numMaps > 0) {
+      infoBlock._("Average Map Time", StringUtils.formatTime(avgMapTime));
+    }
+    if(numReduces > 0) {
+      infoBlock._("Average Reduce Time", StringUtils.formatTime(avgReduceTime));
+      infoBlock._("Average Shuffle Time", StringUtils.formatTime(avgShuffleTime));
+      infoBlock._("Average Merge Time", StringUtils.formatTime(avgSortTime));
+    }
+
+    for(Map.Entry<JobACL, AccessControlList> entry : acls.entrySet()) {
+      infoBlock._("ACL "+entry.getKey().getAclName()+":",
+          entry.getValue().getAclString());
+    }
     html.
       _(InfoBlock.class).
       div(_INFO_WRAP).
@@ -109,34 +135,17 @@ public class HsJobBlock extends HtmlBlock {
         table("#job").
           tr().
             th(_TH, "Task Type").
-            th(_TH, "Progress").
             th(_TH, "Total").
-            th(_TH, "Pending").
-            th(_TH, "Running").
             th(_TH, "Complete")._().
           tr(_ODD).
             th().
               a(url("tasks", jid, "m"), "Map")._().
-            td().
-              div(_PROGRESSBAR).
-                $title(join(mapPct, '%')). // tooltip
-                div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", mapPct, '%'))._()._()._().
             td(String.valueOf(mapTasks)).
-            td(String.valueOf(pendingMapTasks)).
-            td(String.valueOf(runningMapTasks)).
             td(String.valueOf(mapTasksComplete))._().
           tr(_EVEN).
             th().
               a(url("tasks", jid, "r"), "Reduce")._().
-            td().
-              div(_PROGRESSBAR).
-                $title(join(reducePct, '%')). // tooltip
-                div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", reducePct, '%'))._()._()._().
             td(String.valueOf(reduceTasks)).
-            td(String.valueOf(pendingReduceTasks)).
-            td(String.valueOf(runningReduceTasks)).
             td(String.valueOf(reducesTasksComplete))._()
           ._().
 
@@ -144,19 +153,11 @@ public class HsJobBlock extends HtmlBlock {
         table("#job").
         tr().
           th(_TH, "Attempt Type").
-          th(_TH, "New").
-          th(_TH, "Running").
           th(_TH, "Failed").
           th(_TH, "Killed").
           th(_TH, "Successful")._().
         tr(_ODD).
           th("Maps").
-          td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.NEW.toString()), 
-              String.valueOf(newMapAttempts))._().
-          td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.RUNNING.toString()), 
-              String.valueOf(runningMapAttempts))._().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.FAILED.toString()), 
               String.valueOf(failedMapAttempts))._().
@@ -169,12 +170,6 @@ public class HsJobBlock extends HtmlBlock {
         _().
         tr(_EVEN).
           th("Reduces").
-          td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.NEW.toString()), 
-              String.valueOf(newReduceAttempts))._().
-          td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.RUNNING.toString()), 
-              String.valueOf(runningReduceAttempts))._().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.FAILED.toString()), 
               String.valueOf(failedReduceAttempts))._().
@@ -195,44 +190,21 @@ public class HsJobBlock extends HtmlBlock {
    * @param job the job to get counts for.
    */
   private void countTasksAndAttempts(Job job) {
+    numReduces = 0;
+    numMaps = 0;
     Map<TaskId, Task> tasks = job.getTasks();
     for (Task task : tasks.values()) {
-      switch (task.getType()) {
-      case MAP:
-        // Task counts
-        switch (task.getState()) {
-        case RUNNING:
-          ++runningMapTasks;
-          break;
-        case SCHEDULED:
-          ++pendingMapTasks;
-          break;
-        }
-        break;
-      case REDUCE:
-        // Task counts
-        switch (task.getState()) {
-        case RUNNING:
-          ++runningReduceTasks;
-          break;
-        case SCHEDULED:
-          ++pendingReduceTasks;
-          break;
-        }
-        break;
-      }
-
       // Attempts counts
       Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
       for (TaskAttempt attempt : attempts.values()) {
 
-        int newAttempts = 0, running = 0, successful = 0, failed = 0, killed =0;
+        int successful = 0, failed = 0, killed =0;
 
         if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
-          ++newAttempts;
+          //Do Nothing
         } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
             .getState())) {
-          ++running;
+          //Do Nothing
         } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
             .getState())) {
           ++successful;
@@ -246,21 +218,41 @@ public class HsJobBlock extends HtmlBlock {
 
         switch (task.getType()) {
         case MAP:
-          newMapAttempts += newAttempts;
-          runningMapAttempts += running;
           successfulMapAttempts += successful;
           failedMapAttempts += failed;
           killedMapAttempts += killed;
+          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numMaps++;
+            avgMapTime += (attempt.getFinishTime() -
+                attempt.getLaunchTime());
+          }
           break;
         case REDUCE:
-          newReduceAttempts += newAttempts;
-          runningReduceAttempts += running;
           successfulReduceAttempts += successful;
           failedReduceAttempts += failed;
           killedReduceAttempts += killed;
+          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numReduces++;
+            avgShuffleTime += (attempt.getShuffleFinishTime() - 
+                attempt.getLaunchTime());
+            avgSortTime += attempt.getSortFinishTime() - 
+                attempt.getLaunchTime();
+            avgReduceTime += (attempt.getFinishTime() -
+                attempt.getShuffleFinishTime());
+          }
           break;
         }
       }
     }
+
+    if(numMaps > 0) {
+      avgMapTime = avgMapTime / numMaps;
+    }
+    
+    if(numReduces > 0) {
+      avgReduceTime = avgReduceTime / numReduces;
+      avgShuffleTime = avgShuffleTime / numReduces;
+      avgSortTime = avgSortTime / numReduces;
+    }
   }
 }

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

@@ -52,6 +52,7 @@ public class HsNavBlock 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")._()._();
     }

+ 195 - 35
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTaskPage.java

@@ -18,28 +18,32 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
-import static org.apache.hadoop.yarn.util.StringHelper.percent;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_ID;
 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 java.util.Collection;
 
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
 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;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 import com.google.common.base.Joiner;
@@ -68,50 +72,162 @@ public class HsTaskPage extends HsView {
           h2($(TITLE));
         return;
       }
-      TBODY<TABLE<Hamlet>> tbody = html.
+      TaskType type = null;
+      String symbol = $(TASK_TYPE);
+      if (!symbol.isEmpty()) {
+        type = MRApps.taskType(symbol);
+      } else {
+        type = app.getTask().getType();
+      }
+      
+      TR<THEAD<TABLE<Hamlet>>> headRow = html.
       table("#attempts").
         thead().
-          tr().
+          tr();
+      
+      headRow.
             th(".id", "Attempt").
-            th(".progress", "Progress").
             th(".state", "State").
             th(".node", "node").
-            th(".tsh", "Started").
-            th(".tsh", "Finished").
-            th(".tsh", "Elapsed").
-            th(".note", "Note")._()._().
-        tbody();
+            th(".tsh", "Start Time");
+      
+      if(type == TaskType.REDUCE) {
+        headRow.th("Shuffle Finish Time");
+        headRow.th("Merge Finish Time");
+      }
+      
+      headRow.th("Finish Time"); //Attempt
+      
+      if(type == TaskType.REDUCE) {
+        headRow.th("Elapsed Time Shuffle"); //Attempt
+        headRow.th("Elapsed Time Merge"); //Attempt
+        headRow.th("Elapsed Time Reduce"); //Attempt
+      }
+      headRow.th("Elapsed Time").
+              th(".note", "Note");
+      
+       TBODY<TABLE<Hamlet>> tbody = headRow._()._().tbody();
       for (TaskAttempt ta : getTaskAttempts()) {
         String taid = MRApps.toString(ta.getID());
-        String progress = percent(ta.getProgress());
-        ContainerId containerId = ta.getAssignedContainerID();
 
         String nodeHttpAddr = ta.getNodeHttpAddress();
-        long startTime = ta.getLaunchTime();
-        long finishTime = ta.getFinishTime();
-        long elapsed = Times.elapsed(startTime, finishTime);
-        TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = tbody.
-          tr().
-            td(".id", taid).
-            td(".progress", progress).
-            td(".state", ta.getState().toString()).
+        
+        long attemptStartTime = ta.getLaunchTime();
+        long shuffleFinishTime = -1;
+        long sortFinishTime = -1;
+        long attemptFinishTime = ta.getFinishTime();
+        long elapsedShuffleTime = -1;
+        long elapsedSortTime = -1;
+        long elapsedReduceTime = -1;
+        if(type == TaskType.REDUCE) {
+          shuffleFinishTime = ta.getShuffleFinishTime();
+          sortFinishTime = ta.getSortFinishTime();
+          elapsedShuffleTime =
+              Times.elapsed(attemptStartTime, shuffleFinishTime, false);
+          elapsedSortTime =
+              Times.elapsed(shuffleFinishTime, sortFinishTime, false);
+          elapsedReduceTime =
+              Times.elapsed(sortFinishTime, attemptFinishTime, false); 
+        }
+        long attemptElapsed =
+            Times.elapsed(attemptStartTime, attemptFinishTime, false);
+        int sortId = ta.getID().getId() + (ta.getID().getTaskId().getId() * 10000);
+        
+        TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
+        row.
             td().
-              a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
-        if (containerId != null) {
-          String containerIdStr = ConverterUtils.toString(containerId);
-          nodeTd._(" ").
-            a(".logslink", url("http://", nodeHttpAddr, "yarn", "containerlogs",
-              containerIdStr), "logs");
+              br().$title(String.valueOf(sortId))._(). // sorting
+              _(taid)._().
+            td(ta.getState().toString()).
+            td().a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr)._();
+        
+        row.td().
+          br().$title(String.valueOf(attemptStartTime))._().
+            _(Times.format(attemptStartTime))._();
+
+        if(type == TaskType.REDUCE) {
+          row.td().
+            br().$title(String.valueOf(shuffleFinishTime))._().
+            _(Times.format(shuffleFinishTime))._();
+          row.td().
+          br().$title(String.valueOf(sortFinishTime))._().
+          _(Times.format(sortFinishTime))._();
+        }
+        row.
+            td().
+              br().$title(String.valueOf(attemptFinishTime))._().
+              _(Times.format(attemptFinishTime))._();
+        
+        if(type == TaskType.REDUCE) {
+          row.td().
+            br().$title(String.valueOf(elapsedShuffleTime))._().
+          _(formatTime(elapsedShuffleTime))._();
+          row.td().
+          br().$title(String.valueOf(elapsedSortTime))._().
+        _(formatTime(elapsedSortTime))._();
+          row.td().
+            br().$title(String.valueOf(elapsedReduceTime))._().
+          _(formatTime(elapsedReduceTime))._();
         }
-        nodeTd._().
-          td(".ts", Times.format(startTime)).
-          td(".ts", Times.format(finishTime)).
-          td(".dt", StringUtils.formatTime(elapsed)).
-          td(".note", Joiner.on('\n').join(ta.getDiagnostics()))._();
+        
+        row.
+          td().
+            br().$title(String.valueOf(attemptElapsed))._().
+          _(formatTime(attemptElapsed))._().
+          td(".note", Joiner.on('\n').join(ta.getDiagnostics()));
+        row._();
+      }
+      
+      
+      TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
+      footRow.
+          th().input("search_init").$type(InputType.text).
+              $name("attempt_name").$value("Attempt")._()._().
+          th().input("search_init").$type(InputType.text).
+              $name("attempt_state").$value("State")._()._().
+          th().input("search_init").$type(InputType.text).
+              $name("attempt_node").$value("Node")._()._().
+          th().input("search_init").$type(InputType.text).
+              $name("attempt_start_time").$value("Start Time")._()._();
+      
+      if(type == TaskType.REDUCE) {
+        footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("shuffle_time").$value("Shuffle Time")._()._();
+        footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("merge_time").$value("Merge Time")._()._();
       }
-      tbody._()._();
+      
+      footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("attempt_finish").$value("Finish Time")._()._();
+      
+      if(type == TaskType.REDUCE) {
+        footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
+        footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
+        footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
+      }
+
+      footRow.
+        th().input("search_init").$type(InputType.text).
+            $name("attempt_elapsed").$value("Elapsed Time")._()._().
+        th().input("search_init").$type(InputType.text).
+            $name("note").$value("Note")._()._();
+      
+      footRow._()._()._();
     }
 
+    private String formatTime(long elapsed) {
+      return elapsed < 0 ? "N/A" : StringUtils.formatTime(elapsed);
+    }
+    
     /**
      * @return true if this is a valid request else false.
      */
@@ -138,6 +254,7 @@ public class HsTaskPage extends HsView {
     //Set up the java script and CSS for the attempts table
     set(DATATABLES_ID, "attempts");
     set(initID(DATATABLES, "attempts"), attemptsTableInit());
+    set(postInitID(DATATABLES, "attempts"), attemptsPostTableInit());
     setTableStyles(html, "attempts");
   }
 
@@ -154,6 +271,49 @@ public class HsTaskPage extends HsView {
    * attempts table. 
    */
   private String attemptsTableInit() {
-    return tableInit().append("}").toString();
+    TaskType type = null;
+    String symbol = $(TASK_TYPE);
+    if (!symbol.isEmpty()) {
+      type = MRApps.taskType(symbol);
+    } else {
+      TaskId taskID = MRApps.toTaskID($(TASK_ID));
+      type = taskID.getTaskType();
+    }
+    StringBuilder b = tableInit().
+      append(",aoColumnDefs:[");
+
+    b.append("{'sType':'title-numeric', 'aTargets': [ 0");
+    if(type == TaskType.REDUCE) {
+      b.append(", 7, 8, 9, 10");
+    } else { //MAP
+      b.append(", 5");
+    }
+    b.append(" ] }");
+    b.append("]}");
+    return b.toString();
+  }
+  
+  private String attemptsPostTableInit() {
+    return "var asInitVals = new Array();\n" +
+           "$('tfoot input').keyup( function () \n{"+
+           "  attemptsDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
+           "} );\n"+
+           "$('tfoot input').each( function (i) {\n"+
+           "  asInitVals[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 = asInitVals[$('tfoot input').index(this)];\n"+
+           "  }\n"+
+           "} );\n";
   }
 }

+ 235 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java

@@ -0,0 +1,235 @@
+/**
+* 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.hs.webapp;
+
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskReport;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.util.Times;
+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;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TFOOT;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.THEAD;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+import com.google.inject.Inject;
+
+/**
+ * Render the a table of tasks for a given type.
+ */
+public class HsTasksBlock extends HtmlBlock {
+  final App app;
+
+  @Inject HsTasksBlock(App app) {
+    this.app = app;
+  }
+
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.yarn.webapp.view.HtmlBlock#render(org.apache.hadoop.yarn.webapp.view.HtmlBlock.Block)
+   */
+  @Override protected void render(Block html) {
+    if (app.getJob() == null) {
+      html.
+        h2($(TITLE));
+      return;
+    }
+    TaskType type = null;
+    String symbol = $(TASK_TYPE);
+    if (!symbol.isEmpty()) {
+      type = MRApps.taskType(symbol);
+    }
+    
+    THEAD<TABLE<Hamlet>> thead = html.table("#tasks").thead();
+    //Create the spanning row
+    int attemptColSpan = type == TaskType.REDUCE ? 8 : 3;
+    thead.tr().
+      th().$colspan(5).$class("ui-state-default")._("Task")._().
+      th().$colspan(attemptColSpan).$class("ui-state-default").
+        _("Successful Attempt")._().
+    _();
+    
+    TR<THEAD<TABLE<Hamlet>>> theadRow = thead.
+          tr().
+            th("Name").
+            th("State").
+            th("Start Time").
+            th("Finish Time").
+            th("Elapsed Time").
+            th("Start Time"); //Attempt
+    
+    if(type == TaskType.REDUCE) {
+      theadRow.th("Shuffle Finish Time"); //Attempt
+      theadRow.th("Merge Finish Time"); //Attempt
+    }
+    
+    theadRow.th("Finish Time"); //Attempt
+    
+    if(type == TaskType.REDUCE) {
+      theadRow.th("Elapsed Time Shuffle"); //Attempt
+      theadRow.th("Elapsed Time Merge"); //Attempt
+      theadRow.th("Elapsed Time Reduce"); //Attempt
+    }
+    theadRow.th("Elapsed Time"); //Attempt
+    
+    TBODY<TABLE<Hamlet>> tbody = theadRow._()._().tbody();
+    for (Task task : app.getJob().getTasks().values()) {
+      if (type != null && task.getType() != type) {
+        continue;
+      }
+      String tid = MRApps.toString(task.getID());
+      
+      TaskReport report = task.getReport();
+      long startTime = report.getStartTime();
+      long finishTime = report.getFinishTime();
+      long elapsed = Times.elapsed(startTime, finishTime, false);
+      
+      long attemptStartTime = -1;
+      long shuffleFinishTime = -1;
+      long sortFinishTime = -1;
+      long attemptFinishTime = -1;
+      long elapsedShuffleTime = -1;
+      long elapsedSortTime = -1;;
+      long elapsedReduceTime = -1;
+      long attemptElapsed = -1;
+      TaskAttempt successful = getSuccessfulAttempt(task);
+      if(successful != null) {
+        attemptStartTime = successful.getLaunchTime();
+        attemptFinishTime = successful.getFinishTime();
+        if(type == TaskType.REDUCE) {
+          shuffleFinishTime = successful.getShuffleFinishTime();
+          sortFinishTime = successful.getSortFinishTime();
+          elapsedShuffleTime =
+              Times.elapsed(attemptStartTime, shuffleFinishTime, false);
+          elapsedSortTime =
+              Times.elapsed(shuffleFinishTime, sortFinishTime, false);
+          elapsedReduceTime =
+              Times.elapsed(sortFinishTime, attemptFinishTime, false); 
+        }
+        attemptElapsed =
+            Times.elapsed(attemptStartTime, attemptFinishTime, false);
+      }
+      
+      TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
+      row.
+          td().
+            br().$title(String.valueOf(task.getID().getId()))._(). // sorting
+            a(url("task", tid), tid)._().
+          td(report.getTaskState().toString()).
+          td().
+            br().$title(String.valueOf(startTime))._().
+            _(Times.format(startTime))._().
+          td().
+            br().$title(String.valueOf(finishTime))._().
+            _(Times.format(finishTime))._().
+          td().
+            br().$title(String.valueOf(elapsed))._().
+            _(formatTime(elapsed))._().
+          td().
+            br().$title(String.valueOf(attemptStartTime))._().
+            _(Times.format(attemptStartTime))._();
+      if(type == TaskType.REDUCE) {
+        row.td().
+          br().$title(String.valueOf(shuffleFinishTime))._().
+          _(Times.format(shuffleFinishTime))._();
+        row.td().
+        br().$title(String.valueOf(sortFinishTime))._().
+        _(Times.format(sortFinishTime))._();
+      }
+      row.
+          td().
+            br().$title(String.valueOf(attemptFinishTime))._().
+            _(Times.format(attemptFinishTime))._();
+      
+      if(type == TaskType.REDUCE) {
+        row.td().
+          br().$title(String.valueOf(elapsedShuffleTime))._().
+        _(formatTime(elapsedShuffleTime))._();
+        row.td().
+        br().$title(String.valueOf(elapsedSortTime))._().
+      _(formatTime(elapsedSortTime))._();
+        row.td().
+          br().$title(String.valueOf(elapsedReduceTime))._().
+        _(formatTime(elapsedReduceTime))._();
+      }
+      
+      row.td().
+        br().$title(String.valueOf(attemptElapsed))._().
+        _(formatTime(attemptElapsed))._();
+      row._();
+    }
+    TR<TFOOT<TABLE<Hamlet>>> footRow = tbody._().tfoot().tr();
+    footRow.th().input("search_init").$type(InputType.text).$name("task")
+        .$value("ID")._()._().th().input("search_init").$type(InputType.text)
+        .$name("state").$value("State")._()._().th().input("search_init")
+        .$type(InputType.text).$name("start_time").$value("Start Time")._()._()
+        .th().input("search_init").$type(InputType.text).$name("finish_time")
+        .$value("Finish Time")._()._().th().input("search_init")
+        .$type(InputType.text).$name("elapsed_time").$value("Elapsed Time")._()
+        ._().th().input("search_init").$type(InputType.text)
+        .$name("attempt_start_time").$value("Start Time")._()._();
+    
+    if(type == TaskType.REDUCE) {
+      footRow.th().input("search_init").$type(InputType.text)
+          .$name("shuffle_time").$value("Shuffle Time")._()._();
+      footRow.th().input("search_init").$type(InputType.text)
+          .$name("merge_time").$value("Merge Time")._()._();
+    }
+
+    footRow.th().input("search_init").$type(InputType.text)
+        .$name("attempt_finish").$value("Finish Time")._()._();
+
+    if(type == TaskType.REDUCE) {
+      footRow.th().input("search_init").$type(InputType.text)
+          .$name("elapsed_shuffle_time").$value("Elapsed Shuffle Time")._()._();
+      footRow.th().input("search_init").$type(InputType.text)
+          .$name("elapsed_merge_time").$value("Elapsed Merge Time")._()._();
+      footRow.th().input("search_init").$type(InputType.text)
+          .$name("elapsed_reduce_time").$value("Elapsed Reduce Time")._()._();
+    }
+
+    footRow.th().input("search_init").$type(InputType.text)
+        .$name("attempt_elapsed").$value("Elapsed Time")._()._();
+    
+    footRow._()._()._();
+  }
+
+  private String formatTime(long elapsed) {
+    return elapsed < 0 ? "N/A" : StringUtils.formatTime(elapsed);
+  }
+  
+  private TaskAttempt getSuccessfulAttempt(Task task) {
+    for(TaskAttempt attempt: task.getAttempts().values()) {
+      if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
+        return attempt;
+      }
+    }
+    return null;
+  }
+}

+ 48 - 8
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksPage.java

@@ -18,13 +18,16 @@
 
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
 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.TasksBlock;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.webapp.SubView;
 
 /**
@@ -41,15 +44,16 @@ public class HsTasksPage extends HsView {
     set(DATATABLES_ID, "tasks");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
     set(initID(DATATABLES, "tasks"), tasksTableInit());
+    set(postInitID(DATATABLES, "tasks"), jobsPostTableInit());
     setTableStyles(html, "tasks");
   }
-
+  
   /**
    * The content of this page is the TasksBlock
-   * @return TasksBlock.class
+   * @return HsTasksBlock.class
    */
   @Override protected Class<? extends SubView> content() {
-    return TasksBlock.class;
+    return HsTasksBlock.class;
   }
 
   /**
@@ -57,9 +61,45 @@ public class HsTasksPage extends HsView {
    * for the tasks table.
    */
   private String tasksTableInit() {
-    return tableInit().
-        append(",aoColumns:[{sType:'title-numeric'},{sType:'title-numeric',").
-        append("bSearchable:false},null,{sType:'title-numeric'},").
-        append("{sType:'title-numeric'},{sType:'title-numeric'}]}").toString();
+    TaskType type = null;
+    String symbol = $(TASK_TYPE);
+    if (!symbol.isEmpty()) {
+      type = MRApps.taskType(symbol);
+    }
+    StringBuilder b = tableInit().
+    append(",aoColumnDefs:[");
+    b.append("{'sType':'title-numeric', 'aTargets': [ 0, 4");
+    if(type == TaskType.REDUCE) {
+      b.append(", 9, 10, 11, 12");
+    } else { //MAP
+      b.append(", 7");
+    }
+    b.append(" ] }");
+    b.append("]}");
+    return b.toString();
+  }
+  
+  private String jobsPostTableInit() {
+    return "var asInitVals = new Array();\n" +
+           "$('tfoot input').keyup( function () \n{"+
+           "  tasksDataTable.fnFilter( this.value, $('tfoot input').index(this) );\n"+
+           "} );\n"+
+           "$('tfoot input').each( function (i) {\n"+
+           "  asInitVals[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 = asInitVals[$('tfoot input').index(this)];\n"+
+           "  }\n"+
+           "} );\n";
   }
 }

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

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

+ 55 - 7
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestHSWebApp.java

@@ -19,25 +19,36 @@
 package org.apache.hadoop.mapreduce.v2.hs.webapp;
 
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.ATTEMPT_STATE;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.JOB_ID;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
 import static org.junit.Assert.assertEquals;
 
+import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 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.webapp.AMParams;
+import org.apache.hadoop.mapreduce.v2.app.webapp.TestAMWebApp;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.junit.Test;
 
 import com.google.inject.Injector;
 
 public class TestHSWebApp {
+  private static final Log LOG = LogFactory.getLog(TestHSWebApp.class);
 
   static class TestAppContext implements AppContext {
     final ApplicationAttemptId appAttemptID;
@@ -111,16 +122,53 @@ public class TestHSWebApp {
   }
 
   @Test public void testJobView() {
-    WebAppTests.testPage(HsJobPage.class, AppContext.class, new TestAppContext());
+    LOG.info("HsJobPage");
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = TestAMWebApp.getJobParams(appContext);
+    WebAppTests.testPage(HsJobPage.class, AppContext.class, appContext, params);
   }
 
-  @Test public void testTasksView() {
-    WebAppTests.testPage(HsTasksPage.class, AppContext.class,
-                         new TestAppContext());
+  @Test
+  public void testTasksView() {
+    LOG.info("HsTasksPage");
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
+    WebAppTests.testPage(HsTasksPage.class, AppContext.class, appContext,
+        params);
+  }
+
+  @Test
+  public void testTaskView() {
+    LOG.info("HsTaskPage");
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
+    WebAppTests
+        .testPage(HsTaskPage.class, AppContext.class, appContext, params);
   }
 
-  @Test public void testTaskView() {
-    WebAppTests.testPage(HsTaskPage.class, AppContext.class,
+  @Test public void testAttemptsWithJobView() {
+    LOG.info("HsAttemptsPage with data");
+    TestAppContext ctx = new TestAppContext();
+    JobId id = ctx.getAllJobs().keySet().iterator().next();
+    Map<String, String> params = new HashMap<String,String>();
+    params.put(JOB_ID, id.toString());
+    params.put(TASK_TYPE, "m");
+    params.put(ATTEMPT_STATE, "SUCCESSFUL");
+    WebAppTests.testPage(HsAttemptsPage.class, AppContext.class,
+        ctx, params);
+  }
+  
+  @Test public void testAttemptsView() {
+    LOG.info("HsAttemptsPage");
+    AppContext appContext = new TestAppContext();
+    Map<String, String> params = TestAMWebApp.getTaskParams(appContext);
+    WebAppTests.testPage(HsAttemptsPage.class, AppContext.class,
+                         appContext, params);
+  }
+  
+  @Test public void testConfView() {
+    LOG.info("HsConfPage");
+    WebAppTests.testPage(HsConfPage.class, AppContext.class,
                          new TestAppContext());
   }
 }

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

@@ -19,6 +19,7 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
+import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -28,12 +29,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.JobID;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
+import org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
 
 public class ClientCache {
 
@@ -72,16 +74,21 @@ public class ClientCache {
 
   private MRClientProtocol instantiateHistoryProxy()
   throws IOException {
-	String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
+	final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
 	          JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
     LOG.info("Connecting to HistoryServer at: " + serviceAddr);
-    Configuration myConf = new Configuration(conf);
-    //TODO This should ideally be using it's own class (instead of ClientRMSecurityInfo)
+    final Configuration myConf = new Configuration(conf);
     myConf.setClass(YarnConfiguration.YARN_SECURITY_INFO,
-        ClientRMSecurityInfo.class, SecurityInfo.class);
-    YarnRPC rpc = YarnRPC.create(myConf);
+        ClientHSSecurityInfo.class, SecurityInfo.class);
+    final YarnRPC rpc = YarnRPC.create(myConf);
     LOG.info("Connected to HistoryServer at: " + serviceAddr);
-    return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
-        NetUtils.createSocketAddr(serviceAddr), myConf);
+    UserGroupInformation currentUser = UserGroupInformation.getCurrentUser();
+    return currentUser.doAs(new PrivilegedAction<MRClientProtocol>() {
+      @Override
+      public MRClientProtocol run() {
+        return (MRClientProtocol) rpc.getProxy(MRClientProtocol.class,
+            NetUtils.createSocketAddr(serviceAddr), myConf);
+      }
+    });
   }
 }

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

@@ -19,7 +19,6 @@
 package org.apache.hadoop.mapred;
 
 import java.io.IOException;
-import java.net.URI;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -33,7 +32,6 @@ 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.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -55,7 +53,6 @@ import org.apache.hadoop.mapreduce.TaskReport;
 import org.apache.hadoop.mapreduce.TaskTrackerInfo;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TypeConverter;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.MRConstants;
@@ -72,6 +69,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationState;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -237,7 +235,6 @@ public class YARNRunner implements ClientProtocol {
     // Construct necessary information to start the MR AM
     ApplicationSubmissionContext appContext = 
       createApplicationSubmissionContext(conf, jobSubmitDir, ts);
-    setupDistributedCache(conf, appContext);
     
     // XXX Remove
     in.close();
@@ -273,16 +270,18 @@ public class YARNRunner implements ClientProtocol {
   public ApplicationSubmissionContext createApplicationSubmissionContext(
       Configuration jobConf,
       String jobSubmitDir, Credentials ts) throws IOException {
-    ApplicationSubmissionContext appContext =
-        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     ApplicationId applicationId = resMgrDelegate.getApplicationId();
-    appContext.setApplicationId(applicationId);
+    
+    // Setup resource requirements
     Resource capability = recordFactory.newRecordInstance(Resource.class);
     capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
         MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
     LOG.info("AppMaster capability = " + capability);
-    appContext.setMasterCapability(capability);
 
+    // Setup LocalResources
+    Map<String, LocalResource> localResources =
+        new HashMap<String, LocalResource>();
+    
     Path jobConfPath = new Path(jobSubmitDir, MRConstants.JOB_CONF_FILE);
     
     URL yarnUrlForJobSubmitDir = ConverterUtils
@@ -292,14 +291,11 @@ public class YARNRunner implements ClientProtocol {
     LOG.debug("Creating setup context, jobSubmitDir url is "
         + yarnUrlForJobSubmitDir);
 
-    appContext.setResource(MRConstants.JOB_SUBMIT_DIR,
-        yarnUrlForJobSubmitDir);
-
-    appContext.setResourceTodo(MRConstants.JOB_CONF_FILE,
+    localResources.put(MRConstants.JOB_CONF_FILE,
         createApplicationResource(defaultFileContext,
             jobConfPath));
     if (jobConf.get(MRJobConfig.JAR) != null) {
-      appContext.setResourceTodo(MRConstants.JOB_JAR,
+      localResources.put(MRConstants.JOB_JAR,
           createApplicationResource(defaultFileContext,
               new Path(jobSubmitDir, MRConstants.JOB_JAR)));
     } else {
@@ -312,30 +308,21 @@ public class YARNRunner implements ClientProtocol {
     // TODO gross hack
     for (String s : new String[] { "job.split", "job.splitmetainfo",
         MRConstants.APPLICATION_TOKENS_FILE }) {
-      appContext.setResourceTodo(
+      localResources.put(
           MRConstants.JOB_SUBMIT_DIR + "/" + s,
-          createApplicationResource(defaultFileContext, new Path(jobSubmitDir, s)));
+          createApplicationResource(defaultFileContext, 
+              new Path(jobSubmitDir, s)));
     }
-
-    // TODO: Only if security is on.
-    List<String> fsTokens = new ArrayList<String>();
-    for (Token<? extends TokenIdentifier> token : ts.getAllTokens()) {
-      fsTokens.add(token.encodeToUrlString());
-    }
-    
-    // TODO - Remove this!
-    appContext.addAllFsTokens(fsTokens);
-    DataOutputBuffer dob = new DataOutputBuffer();
-    ts.writeTokenStorageToStream(dob);
-    appContext.setFsTokensTodo(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
-
-    // Add queue information
-    appContext.setQueue(jobConf.get(JobContext.QUEUE_NAME, JobConf.DEFAULT_QUEUE_NAME));
-    
-    // Add job name
-    appContext.setApplicationName(jobConf.get(JobContext.JOB_NAME, "N/A"));
     
-    // Add the command line
+    // Setup security tokens
+    ByteBuffer securityTokens = null;
+    if (UserGroupInformation.isSecurityEnabled()) {
+      DataOutputBuffer dob = new DataOutputBuffer();
+      ts.writeTokenStorageToStream(dob);
+      securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
+    }
+
+    // Setup the command to run the AM
     String javaHome = "$JAVA_HOME";
     Vector<CharSequence> vargs = new Vector<CharSequence>(8);
     vargs.add(javaHome + "/bin/java");
@@ -346,13 +333,6 @@ public class YARNRunner implements ClientProtocol {
     vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
         MRJobConfig.DEFAULT_MR_AM_COMMAND_OPTS));
 
-    // Add { job jar, MR app jar } to classpath.
-    Map<String, String> environment = new HashMap<String, String>();
-    MRApps.setInitialClasspath(environment);
-    MRApps.addToClassPath(environment, MRConstants.JOB_JAR);
-    MRApps.addToClassPath(environment,
-        MRConstants.YARN_MAPREDUCE_APP_JAR_PATH);
-    appContext.addAllEnvironment(environment);
     vargs.add("org.apache.hadoop.mapreduce.v2.app.MRAppMaster");
     vargs.add(String.valueOf(applicationId.getClusterTimestamp()));
     vargs.add(String.valueOf(applicationId.getId()));
@@ -370,140 +350,43 @@ public class YARNRunner implements ClientProtocol {
 
     LOG.info("Command to launch container for ApplicationMaster is : "
         + mergedCommand);
+    
+    // Setup the environment - Add { job jar, MR app jar } to classpath.
+    Map<String, String> environment = new HashMap<String, String>();
+    MRApps.setInitialClasspath(environment);
+    MRApps.addToClassPath(environment, MRConstants.JOB_JAR);
+    MRApps.addToClassPath(environment,
+        MRConstants.YARN_MAPREDUCE_APP_JAR_PATH);
 
-    appContext.addAllCommands(vargsFinal);
-    // TODO: RM should get this from RPC.
-    appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName());
-    return appContext;
-  }
+    // Parse distributed cache
+    MRApps.setupDistributedCache(jobConf, localResources, environment);
 
-  /**
-   *    * TODO: Copied for now from TaskAttemptImpl.java ... fixme
-   * @param strs
-   * @return
-   */
-  private static long[] parseTimeStamps(String[] strs) {
-    if (null == strs) {
-      return null;
-    }
-    long[] result = new long[strs.length];
-    for(int i=0; i < strs.length; ++i) {
-      result[i] = Long.parseLong(strs[i]);
-    }
-    return result;
-  }
+    // Setup ContainerLaunchContext for AM container
+    ContainerLaunchContext amContainer =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    amContainer.setResource(capability);             // Resource (mem) required
+    amContainer.setLocalResources(localResources);   // Local resources
+    amContainer.setEnvironment(environment);         // Environment
+    amContainer.setCommands(vargsFinal);             // Command for AM
+    amContainer.setContainerTokens(securityTokens);  // Security tokens
 
-  /**
-   * TODO: Copied for now from TaskAttemptImpl.java ... fixme
-   * 
-   * TODO: This is currently needed in YarnRunner as user code like setupJob,
-   * cleanupJob may need access to dist-cache. Once we separate distcache for
-   * maps, reduces, setup etc, this can include only a subset of artificats.
-   * This is also needed for uberAM case where we run everything inside AM.
-   */
-  private void setupDistributedCache(Configuration conf, 
-      ApplicationSubmissionContext container) throws IOException {
-    
-    // Cache archives
-    parseDistributedCacheArtifacts(conf, container, LocalResourceType.ARCHIVE, 
-        DistributedCache.getCacheArchives(conf), 
-        parseTimeStamps(DistributedCache.getArchiveTimestamps(conf)), 
-        getFileSizes(conf, MRJobConfig.CACHE_ARCHIVES_SIZES), 
-        DistributedCache.getArchiveVisibilities(conf), 
-        DistributedCache.getArchiveClassPaths(conf));
-    
-    // Cache files
-    parseDistributedCacheArtifacts(conf, container, LocalResourceType.FILE, 
-        DistributedCache.getCacheFiles(conf),
-        parseTimeStamps(DistributedCache.getFileTimestamps(conf)),
-        getFileSizes(conf, MRJobConfig.CACHE_FILES_SIZES),
-        DistributedCache.getFileVisibilities(conf),
-        DistributedCache.getFileClassPaths(conf));
-  }
-
-  // TODO - Move this to MR!
-  // Use TaskDistributedCacheManager.CacheFiles.makeCacheFiles(URI[], long[], boolean[], Path[], FileType)
-  private void parseDistributedCacheArtifacts(Configuration conf,
-      ApplicationSubmissionContext container, LocalResourceType type,
-      URI[] uris, long[] timestamps, long[] sizes, boolean visibilities[], 
-      Path[] pathsToPutOnClasspath) throws IOException {
-
-    if (uris != null) {
-      // Sanity check
-      if ((uris.length != timestamps.length) || (uris.length != sizes.length) ||
-          (uris.length != visibilities.length)) {
-        throw new IllegalArgumentException("Invalid specification for " +
-            "distributed-cache artifacts of type " + type + " :" +
-            " #uris=" + uris.length +
-            " #timestamps=" + timestamps.length +
-            " #visibilities=" + visibilities.length
-            );
-      }
-      
-      Map<String, Path> classPaths = new HashMap<String, Path>();
-      if (pathsToPutOnClasspath != null) {
-        for (Path p : pathsToPutOnClasspath) {
-          FileSystem fs = p.getFileSystem(conf);
-          p = p.makeQualified(fs.getUri(), fs.getWorkingDirectory());
-          classPaths.put(p.toUri().getPath().toString(), p);
-        }
-      }
-      for (int i = 0; i < uris.length; ++i) {
-        URI u = uris[i];
-        Path p = new Path(u);
-        FileSystem fs = p.getFileSystem(conf);
-        p = fs.resolvePath(
-            p.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
-        // Add URI fragment or just the filename
-        Path name = new Path((null == u.getFragment())
-          ? p.getName()
-          : u.getFragment());
-        if (name.isAbsolute()) {
-          throw new IllegalArgumentException("Resource name must be relative");
-        }
-        String linkName = name.toUri().getPath();
-        container.setResourceTodo(
-            linkName,
-            createLocalResource(
-                p.toUri(), type, 
-                visibilities[i]
-                  ? LocalResourceVisibility.PUBLIC
-                  : LocalResourceVisibility.PRIVATE,
-                sizes[i], timestamps[i])
-        );
-        if (classPaths.containsKey(u.getPath())) {
-          Map<String, String> environment = container.getAllEnvironment();
-          MRApps.addToClassPath(environment, linkName);
-        }
-      }
-    }
-  }
+    // Set up the ApplicationSubmissionContext
+    ApplicationSubmissionContext appContext =
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    appContext.setApplicationId(applicationId);                // ApplicationId
+    appContext.setUser(                                        // User name
+        UserGroupInformation.getCurrentUser().getShortUserName());
+    appContext.setQueue(                                       // Queue name
+        jobConf.get(JobContext.QUEUE_NAME,     
+        YarnConfiguration.DEFAULT_QUEUE_NAME));
+    appContext.setApplicationName(                             // Job name
+        jobConf.get(JobContext.JOB_NAME, 
+        YarnConfiguration.DEFAULT_APPLICATION_NAME));              
+    appContext.setAMContainerSpec(amContainer);         // AM Container 
 
-  // TODO - Move this to MR!
-  private static long[] getFileSizes(Configuration conf, String key) {
-    String[] strs = conf.getStrings(key);
-    if (strs == null) {
-      return null;
-    }
-    long[] result = new long[strs.length];
-    for(int i=0; i < strs.length; ++i) {
-      result[i] = Long.parseLong(strs[i]);
-    }
-    return result;
-  }
-  
-  private LocalResource createLocalResource(URI uri, 
-      LocalResourceType type, LocalResourceVisibility visibility, 
-      long size, long timestamp) throws IOException {
-    LocalResource resource = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(LocalResource.class);
-    resource.setResource(ConverterUtils.getYarnUrlFromURI(uri));
-    resource.setType(type);
-    resource.setVisibility(visibility);
-    resource.setSize(size);
-    resource.setTimestamp(timestamp);
-    return resource;
+    return appContext;
   }
-  
+
   @Override
   public void setJobPriority(JobID arg0, String arg1) throws IOException,
       InterruptedException {

+ 1 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/resources/META-INF/services/org.apache.hadoop.security.SecurityInfo

@@ -0,0 +1 @@
+org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo

+ 15 - 203
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java

@@ -18,14 +18,8 @@
 
 package org.apache.hadoop.yarn.api.records;
 
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 
 /**
@@ -36,26 +30,17 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>It includes details such as:
  *   <ul>
  *     <li>{@link ApplicationId} of the application.</li>
- *     <li>
- *       {@link Resource} necessary to run the <code>ApplicationMaster</code>.
- *     </li>
  *     <li>Application user.</li>
  *     <li>Application name.</li>
  *     <li>{@link Priority} of the application.</li>
- *     <li>Security tokens (if security is enabled).</li>
- *     <li>
- *       {@link LocalResource} necessary for running the 
- *       <code>ApplicationMaster</code> container such
- *       as binaries, jar, shared-objects, side-files etc. 
- *     </li>
  *     <li>
- *       Environment variables for the launched <code>ApplicationMaster</code> 
- *       process.
+ *       {@link ContainerLaunchContext} of the container in which the 
+ *       <code>ApplicationMaster</code> is executed.
  *     </li>
- *     <li>Command to launch the <code>ApplicationMaster</code>.</li>
  *   </ul>
  * </p>
  * 
+ * @see ContainerLaunchContext
  * @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)
  */
 @Public
@@ -143,198 +128,25 @@ public interface ApplicationSubmissionContext {
   public void setUser(String user);
   
   /**
-   * Get the <code>Resource</code> required to run the 
-   * <code>ApplicationMaster</code>.
-   * @return <code>Resource</code> required to run the 
-   *         <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public Resource getMasterCapability();
-  
-  /**
-   * Set <code>Resource</code> required to run the 
-   * <code>ApplicationMaster</code>.
-   * @param masterCapability <code>Resource</code> required to run the 
-   *                         <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public void setMasterCapability(Resource masterCapability);
-  
-  @Private
-  @Unstable
-  public Map<String, URL> getAllResources();
-  
-  @Private
-  @Unstable
-  public URL getResource(String key);
-  
-  @Private
-  @Unstable
-  public void addAllResources(Map<String, URL> resources);
-
-  @Private
-  @Unstable
-  public void setResource(String key, URL url);
-
-  @Private
-  @Unstable
-  public void removeResource(String key);
-
-  @Private
-  @Unstable
-  public void clearResources();
-
-  /**
-   * Get all the <code>LocalResource</code> required to run the 
-   * <code>ApplicationMaster</code>.
-   * @return <code>LocalResource</code> required to run the 
-   *         <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public Map<String, LocalResource> getAllResourcesTodo();
-  
-  @Private
-  @Unstable
-  public LocalResource getResourceTodo(String key);
-  
-  /**
-   * Add all the <code>LocalResource</code> required to run the 
-   * <code>ApplicationMaster</code>.
-   * @param resources all <code>LocalResource</code> required to run the 
-   *                      <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public void addAllResourcesTodo(Map<String, LocalResource> resources);
-
-  @Private
-  @Unstable
-  public void setResourceTodo(String key, LocalResource localResource);
-
-  @Private
-  @Unstable
-  public void removeResourceTodo(String key);
-
-  @Private
-  @Unstable
-  public void clearResourcesTodo();
-
-  @Private
-  @Unstable
-  public List<String> getFsTokenList();
-  
-  @Private
-  @Unstable
-  public String getFsToken(int index);
-  
-  @Private
-  @Unstable
-  public int getFsTokenCount();
-  
-  @Private
-  @Unstable
-  public void addAllFsTokens(List<String> fsTokens);
-
-  @Private
-  @Unstable
-  public void addFsToken(String fsToken);
-
-  @Private
-  @Unstable
-  public void removeFsToken(int index);
-
-  @Private
-  @Unstable
-  public void clearFsTokens();
-
-  /**
-   * Get <em>file-system tokens</em> for the <code>ApplicationMaster</code>.
-   * @return file-system tokens for the <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public ByteBuffer getFsTokensTodo();
-  
-  /**
-   * Set <em>file-system tokens</em> for the <code>ApplicationMaster</code>.
-   * @param fsTokens file-system tokens for the <code>ApplicationMaster</code>
+   * Get the <code>ContainerLaunchContext</code> to describe the 
+   * <code>Container</code> with which the <code>ApplicationMaster</code> is
+   * launched.
+   * @return <code>ContainerLaunchContext</code> for the 
+   *         <code>ApplicationMaster</code> container
    */
   @Public
   @Stable
-  public void setFsTokensTodo(ByteBuffer fsTokens);
-
-  /**
-   * Get the <em>environment variables</em> for the 
-   * <code>ApplicationMaster</code>.
-   * @return environment variables for the <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public Map<String, String> getAllEnvironment();
-  
-  @Private
-  @Unstable
-  public String getEnvironment(String key);
+  public ContainerLaunchContext getAMContainerSpec();
   
   /**
-   * Add all of the <em>environment variables</em> for the 
-   * <code>ApplicationMaster</code>.
-   * @param environment environment variables for the 
-   *                    <code>ApplicationMaster</code>
+   * Set the <code>ContainerLaunchContext</code> to describe the 
+   * <code>Container</code> with which the <code>ApplicationMaster</code> is
+   * launched.
+   * @param amContainer <code>ContainerLaunchContext</code> for the 
+   *                    <code>ApplicationMaster</code> container
    */
   @Public
   @Stable
-  public void addAllEnvironment(Map<String, String> environment);
+  public void setAMContainerSpec(ContainerLaunchContext amContainer);
 
-  @Private
-  @Unstable
-  public void setEnvironment(String key, String env);
-
-  @Private
-  @Unstable
-  public void removeEnvironment(String key);
-
-  @Private
-  @Unstable
-  public void clearEnvironment();
-
-  /**
-   * Get the <em>commands</em> to launch the <code>ApplicationMaster</code>.
-   * @return commands to launch the <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public List<String> getCommandList();
-  
-  @Private
-  @Unstable
-  public String getCommand(int index);
-  
-  @Private
-  @Unstable
-  public int getCommandCount();
-  
-  /**
-   * Add all of the <em>commands</em> to launch the 
-   * <code>ApplicationMaster</code>.
-   * @param commands commands to launch the <code>ApplicationMaster</code>
-   */
-  @Public
-  @Stable
-  public void addAllCommands(List<String> commands);
-  
-  @Private
-  @Unstable
-  public void addCommand(String command);
-  
-  @Private
-  @Unstable
-  public void removeCommand(int index);
-  
-  @Private
-  @Unstable
-  public void clearCommands();
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java

@@ -156,7 +156,7 @@ public interface ContainerLaunchContext {
    */
   @Public
   @Stable
-  Map<String, String> getEnv();
+  Map<String, String> getEnvironment();
     
   /**
    * Add <em>environment variables</em> for the container.
@@ -164,7 +164,7 @@ public interface ContainerLaunchContext {
    */
   @Public
   @Stable
-  void setEnv(Map<String, String> environment);
+  void setEnvironment(Map<String, String> environment);
 
   /**
    * Get the list of <em>commands</em> for launching the container.

+ 43 - 469
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java

@@ -18,56 +18,35 @@
 
 package org.apache.hadoop.yarn.api.records.impl.pb;
 
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
-import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.StringURLMapProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
-
-
     
-public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSubmissionContextProto> implements ApplicationSubmissionContext {
-  ApplicationSubmissionContextProto proto = ApplicationSubmissionContextProto.getDefaultInstance();
+public class ApplicationSubmissionContextPBImpl 
+extends ProtoBase<ApplicationSubmissionContextProto> 
+implements ApplicationSubmissionContext {
+  ApplicationSubmissionContextProto proto = 
+      ApplicationSubmissionContextProto.getDefaultInstance();
   ApplicationSubmissionContextProto.Builder builder = null;
   boolean viaProto = false;
   
   private ApplicationId applicationId = null;
-  private Resource masterCapability = null;
-  private Map<String, URL> resources = null;
-  private Map<String, LocalResource> resourcesTodo = null;
-  private List<String> fsTokenList = null;
-  private ByteBuffer fsTokenTodo = null;
-  private Map<String, String> environment = null;
-  private List<String> commandList = null;
   private Priority priority = null;
-  
-  
+  private ContainerLaunchContext amContainer = null;
   
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
   }
 
-  public ApplicationSubmissionContextPBImpl(ApplicationSubmissionContextProto proto) {
+  public ApplicationSubmissionContextPBImpl(
+      ApplicationSubmissionContextProto proto) {
     this.proto = proto;
     viaProto = true;
   }
@@ -83,30 +62,12 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     if (this.applicationId != null) {
       builder.setApplicationId(convertToProtoFormat(this.applicationId));
     }
-    if (this.masterCapability != null) {
-      builder.setMasterCapability(convertToProtoFormat(this.masterCapability));
-    }
-    if (this.resources != null) {
-      addResourcesToProto();
-    }
-    if (this.resourcesTodo != null) {
-      addResourcesTodoToProto();
-    }
-    if (this.fsTokenList != null) {
-      addFsTokenListToProto();
-    }
-    if (this.fsTokenTodo != null) {
-      builder.setFsTokensTodo(convertToProtoFormat(this.fsTokenTodo));
-    }
-    if (this.environment != null) {
-      addEnvironmentToProto();
-    }
-    if (this.commandList != null) {
-      addCommandsToProto();
-    }
     if (this.priority != null) {
       builder.setPriority(convertToProtoFormat(this.priority));
     }
+    if (this.amContainer != null) {
+      builder.setAmContainerSpec(convertToProtoFormat(this.amContainer));
+    }
   }
 
   private void mergeLocalToProto() {
@@ -145,6 +106,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
       builder.clearPriority();
     this.priority = priority;
   }
+  
   @Override
   public ApplicationId getApplicationId() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -165,6 +127,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
       builder.clearApplicationId();
     this.applicationId = applicationId;
   }
+  
   @Override
   public String getApplicationName() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -183,403 +146,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     }
     builder.setApplicationName((applicationName));
   }
-  @Override
-  public Resource getMasterCapability() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.masterCapability != null) {
-      return masterCapability;
-    } // Else via proto
-    if (!p.hasMasterCapability()) {
-      return null;
-    }
-    masterCapability = convertFromProtoFormat(p.getMasterCapability());
-    return this.masterCapability;
-  }
-
-  @Override
-  public void setMasterCapability(Resource masterCapability) {
-    maybeInitBuilder();
-    if (masterCapability == null)
-      builder.clearMasterCapability();
-    this.masterCapability = masterCapability;
-  }
-  @Override
-  public Map<String, URL> getAllResources() {
-    initResources();
-    return this.resources;
-  }
-  @Override
-  public URL getResource(String key) {
-    initResources();
-    return this.resources.get(key);
-  }
-  
-  private void initResources() {
-    if (this.resources != null) {
-      return;
-    }
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<StringURLMapProto> mapAsList = p.getResourcesList();
-    this.resources = new HashMap<String, URL>();
-    
-    for (StringURLMapProto c : mapAsList) {
-      this.resources.put(c.getKey(), convertFromProtoFormat(c.getValue()));
-    }
-  }
-  
-  @Override
-  public void addAllResources(final Map<String, URL> resources) {
-    if (resources == null)
-      return;
-    initResources();
-    this.resources.putAll(resources);
-  }
-  
-  private void addResourcesToProto() {
-    maybeInitBuilder();
-    builder.clearResources();
-    if (this.resources == null)
-      return;
-    Iterable<StringURLMapProto> iterable = new Iterable<StringURLMapProto>() {
-      
-      @Override
-      public Iterator<StringURLMapProto> iterator() {
-        return new Iterator<StringURLMapProto>() {
-          
-          Iterator<String> keyIter = resources.keySet().iterator();
-          
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
-          }
-          
-          @Override
-          public StringURLMapProto next() {
-            String key = keyIter.next();
-            return StringURLMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(resources.get(key))).build();
-          }
-          
-          @Override
-          public boolean hasNext() {
-            return keyIter.hasNext();
-          }
-        };
-      }
-    };
-    builder.addAllResources(iterable);
-  }
-  @Override
-  public void setResource(String key, URL val) {
-    initResources();
-    this.resources.put(key, val);
-  }
-  @Override
-  public void removeResource(String key) {
-    initResources();
-    this.resources.remove(key);
-  }
-  @Override
-  public void clearResources() {
-    initResources();
-    this.resources.clear();
-  }
-  @Override
-  public Map<String, LocalResource> getAllResourcesTodo() {
-    initResourcesTodo();
-    return this.resourcesTodo;
-  }
-  @Override
-  public LocalResource getResourceTodo(String key) {
-    initResourcesTodo();
-    return this.resourcesTodo.get(key);
-  }
-  
-  private void initResourcesTodo() {
-    if (this.resourcesTodo != null) {
-      return;
-    }
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<StringLocalResourceMapProto> mapAsList = p.getResourcesTodoList();
-    this.resourcesTodo = new HashMap<String, LocalResource>();
-    
-    for (StringLocalResourceMapProto c : mapAsList) {
-      this.resourcesTodo.put(c.getKey(), convertFromProtoFormat(c.getValue()));
-    }
-  }
-  
-  @Override
-  public void addAllResourcesTodo(final Map<String, LocalResource> resourcesTodo) {
-    if (resourcesTodo == null) 
-      return;
-    initResourcesTodo();
-    this.resourcesTodo.putAll(resourcesTodo);
-  }
-  
-  private void addResourcesTodoToProto() {
-    maybeInitBuilder();
-    builder.clearResourcesTodo();
-    if (resourcesTodo == null)
-      return;
-    Iterable<StringLocalResourceMapProto> iterable = new Iterable<StringLocalResourceMapProto>() {
-      
-      @Override
-      public Iterator<StringLocalResourceMapProto> iterator() {
-        return new Iterator<StringLocalResourceMapProto>() {
-          
-          Iterator<String> keyIter = resourcesTodo.keySet().iterator();
-          
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
-          }
-          
-          @Override
-          public StringLocalResourceMapProto next() {
-            String key = keyIter.next();
-            return StringLocalResourceMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(resourcesTodo.get(key))).build();
-          }
-          
-          @Override
-          public boolean hasNext() {
-            return keyIter.hasNext();
-          }
-        };
-      }
-    };
-    builder.addAllResourcesTodo(iterable);
-  }
-  @Override
-  public void setResourceTodo(String key, LocalResource val) {
-    initResourcesTodo();
-    this.resourcesTodo.put(key, val);
-  }
-  @Override
-  public void removeResourceTodo(String key) {
-    initResourcesTodo();
-    this.resourcesTodo.remove(key);
-  }
-  @Override
-  public void clearResourcesTodo() {
-    initResourcesTodo();
-    this.resourcesTodo.clear();
-  }
-  @Override
-  public List<String> getFsTokenList() {
-    initFsTokenList();
-    return this.fsTokenList;
-  }
-  @Override
-  public String getFsToken(int index) {
-    initFsTokenList();
-    return this.fsTokenList.get(index);
-  }
-  @Override
-  public int getFsTokenCount() {
-    initFsTokenList();
-    return this.fsTokenList.size();
-  }
-  
-  private void initFsTokenList() {
-    if (this.fsTokenList != null) {
-      return;
-    }
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<String> list = p.getFsTokensList();
-    this.fsTokenList = new ArrayList<String>();
-
-    for (String c : list) {
-      this.fsTokenList.add(c);
-    }
-  }
-  
-  @Override
-  public void addAllFsTokens(final List<String> fsTokens) {
-    if (fsTokens == null) 
-      return;
-    initFsTokenList();
-    this.fsTokenList.addAll(fsTokens);
-  }
-  
-  private void addFsTokenListToProto() {
-    maybeInitBuilder();
-    builder.clearFsTokens();
-    builder.addAllFsTokens(this.fsTokenList);
-  }
-
-  @Override
-  public void addFsToken(String fsTokens) {
-    initFsTokenList();
-    this.fsTokenList.add(fsTokens);
-  }
-  @Override
-  public void removeFsToken(int index) {
-    initFsTokenList();
-    this.fsTokenList.remove(index);
-  }
-  @Override
-  public void clearFsTokens() {
-    initFsTokenList();
-    this.fsTokenList.clear();
-  }
-  @Override
-  public ByteBuffer getFsTokensTodo() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.fsTokenTodo != null) {
-      return this.fsTokenTodo;
-    }
-    if (!p.hasFsTokensTodo()) {
-      return null;
-    }
-    this.fsTokenTodo = convertFromProtoFormat(p.getFsTokensTodo());
-    return this.fsTokenTodo;
-  }
 
-  @Override
-  public void setFsTokensTodo(ByteBuffer fsTokensTodo) {
-    maybeInitBuilder();
-    if (fsTokensTodo == null) 
-      builder.clearFsTokensTodo();
-    this.fsTokenTodo = fsTokensTodo;
-  }
-  @Override
-  public Map<String, String> getAllEnvironment() {
-    initEnvironment();
-    return this.environment;
-  }
-  @Override
-  public String getEnvironment(String key) {
-    initEnvironment();
-    return this.environment.get(key);
-  }
-  
-  private void initEnvironment() {
-    if (this.environment != null) {
-      return;
-    }
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<StringStringMapProto> mapAsList = p.getEnvironmentList();
-    this.environment = new HashMap<String, String>();
-    
-    for (StringStringMapProto c : mapAsList) {
-      this.environment.put(c.getKey(), c.getValue());
-    }
-  }
-  
-  @Override
-  public void addAllEnvironment(Map<String, String> environment) {
-    if (environment == null)
-      return;
-    initEnvironment();
-    this.environment.putAll(environment);
-  }
-  
-  private void addEnvironmentToProto() {
-    maybeInitBuilder();
-    builder.clearEnvironment();
-    if (environment == null)
-      return;
-    Iterable<StringStringMapProto> iterable = new Iterable<StringStringMapProto>() {
-      
-      @Override
-      public Iterator<StringStringMapProto> iterator() {
-        return new Iterator<StringStringMapProto>() {
-          
-          Iterator<String> keyIter = environment.keySet().iterator();
-          
-          @Override
-          public void remove() {
-            throw new UnsupportedOperationException();
-          }
-          
-          @Override
-          public StringStringMapProto next() {
-            String key = keyIter.next();
-            return StringStringMapProto.newBuilder().setKey(key).setValue((environment.get(key))).build();
-          }
-          
-          @Override
-          public boolean hasNext() {
-            return keyIter.hasNext();
-          }
-        };
-      }
-    };
-    builder.addAllEnvironment(iterable);
-  }
-  @Override
-  public void setEnvironment(String key, String val) {
-    initEnvironment();
-    this.environment.put(key, val);
-  }
-  @Override
-  public void removeEnvironment(String key) {
-    initEnvironment();
-    this.environment.remove(key);
-  }
-  @Override
-  public void clearEnvironment() {
-    initEnvironment();
-    this.environment.clear();
-  }
-  @Override
-  public List<String> getCommandList() {
-    initCommandList();
-    return this.commandList;
-  }
-  @Override
-  public String getCommand(int index) {
-    initCommandList();
-    return this.commandList.get(index);
-  }
-  @Override
-  public int getCommandCount() {
-    initCommandList();
-    return this.commandList.size();
-  }
-  
-  private void initCommandList() {
-    if (this.commandList != null) {
-      return;
-    }
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<String> list = p.getCommandList();
-    this.commandList = new ArrayList<String>();
-
-    for (String c : list) {
-      this.commandList.add(c);
-    }
-  }
-  
-  @Override
-  public void addAllCommands(final List<String> command) {
-    if (command == null)
-      return;
-    initCommandList();
-    this.commandList.addAll(command);
-  }
-  
-  private void addCommandsToProto() {
-    maybeInitBuilder();
-    builder.clearCommand();
-    if (this.commandList == null) 
-      return;
-    builder.addAllCommand(this.commandList);
-  }
-  @Override
-  public void addCommand(String command) {
-    initCommandList();
-    this.commandList.add(command);
-  }
-  @Override
-  public void removeCommand(int index) {
-    initCommandList();
-    this.commandList.remove(index);
-  }
-  @Override
-  public void clearCommands() {
-    initCommandList();
-    this.commandList.clear();
-  }
   @Override
   public String getQueue() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -598,6 +165,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     }
     builder.setQueue((queue));
   }
+  
   @Override
   public String getUser() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -617,6 +185,28 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     builder.setUser((user));
   }
 
+  @Override
+  public ContainerLaunchContext getAMContainerSpec() {
+    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.amContainer != null) {
+      return amContainer;
+    } // Else via proto
+    if (!p.hasAmContainerSpec()) {
+      return null;
+    }
+    amContainer = convertFromProtoFormat(p.getAmContainerSpec());
+    return amContainer;
+  }
+
+  @Override
+  public void setAMContainerSpec(ContainerLaunchContext amContainer) {
+    maybeInitBuilder();
+    if (amContainer == null) {
+      builder.clearAmContainerSpec();
+    }
+    this.amContainer = amContainer;
+  }
+
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
   }
@@ -633,28 +223,12 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     return ((ApplicationIdPBImpl)t).getProto();
   }
 
-  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl)t).getProto();
-  }
-
-  private URLPBImpl convertFromProtoFormat(URLProto p) {
-    return new URLPBImpl(p);
+  private ContainerLaunchContextPBImpl convertFromProtoFormat(
+      ContainerLaunchContextProto p) {
+    return new ContainerLaunchContextPBImpl(p);
   }
 
-  private URLProto convertToProtoFormat(URL t) {
-    return ((URLPBImpl)t).getProto();
+  private ContainerLaunchContextProto convertToProtoFormat(ContainerLaunchContext t) {
+    return ((ContainerLaunchContextPBImpl)t).getProto();
   }
-
-  private LocalResourcePBImpl convertFromProtoFormat(LocalResourceProto p) {
-    return new LocalResourcePBImpl(p);
-  }
-
-  private LocalResourceProto convertToProtoFormat(LocalResource t) {
-    return ((LocalResourcePBImpl)t).getProto();
-  }
-
 }  

+ 16 - 19
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java

@@ -39,8 +39,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
 
-
-    
 public class ContainerLaunchContextPBImpl 
 extends ProtoBase<ContainerLaunchContextProto> 
 implements ContainerLaunchContext {
@@ -54,10 +52,9 @@ implements ContainerLaunchContext {
   private Map<String, LocalResource> localResources = null;
   private ByteBuffer containerTokens = null;
   private Map<String, ByteBuffer> serviceData = null;
-  private Map<String, String> env = null;
+  private Map<String, String> environment = null;
   private List<String> commands = null;
   
-  
   public ContainerLaunchContextPBImpl() {
     builder = ContainerLaunchContextProto.newBuilder();
   }
@@ -94,7 +91,7 @@ implements ContainerLaunchContext {
     if (this.serviceData != null) {
       addServiceDataToProto();
     }
-    if (this.env != null) {
+    if (this.environment != null) {
       addEnvToProto();
     }
     if (this.commands != null) {
@@ -364,37 +361,37 @@ implements ContainerLaunchContext {
   }
   
   @Override
-  public Map<String, String> getEnv() {
+  public Map<String, String> getEnvironment() {
     initEnv();
-    return this.env;
+    return this.environment;
   }
   
   private void initEnv() {
-    if (this.env != null) {
+    if (this.environment != null) {
       return;
     }
     ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
-    List<StringStringMapProto> list = p.getEnvList();
-    this.env = new HashMap<String, String>();
+    List<StringStringMapProto> list = p.getEnvironmentList();
+    this.environment = new HashMap<String, String>();
 
     for (StringStringMapProto c : list) {
-      this.env.put(c.getKey(), c.getValue());
+      this.environment.put(c.getKey(), c.getValue());
     }
   }
   
   @Override
-  public void setEnv(final Map<String, String> env) {
+  public void setEnvironment(final Map<String, String> env) {
     if (env == null)
       return;
     initEnv();
-    this.env.clear();
-    this.env.putAll(env);
+    this.environment.clear();
+    this.environment.putAll(env);
   }
   
   private void addEnvToProto() {
     maybeInitBuilder();
-    builder.clearEnv();
-    if (env == null)
+    builder.clearEnvironment();
+    if (environment == null)
       return;
     Iterable<StringStringMapProto> iterable = 
         new Iterable<StringStringMapProto>() {
@@ -403,7 +400,7 @@ implements ContainerLaunchContext {
       public Iterator<StringStringMapProto> iterator() {
         return new Iterator<StringStringMapProto>() {
           
-          Iterator<String> keyIter = env.keySet().iterator();
+          Iterator<String> keyIter = environment.keySet().iterator();
           
           @Override
           public void remove() {
@@ -414,7 +411,7 @@ implements ContainerLaunchContext {
           public StringStringMapProto next() {
             String key = keyIter.next();
             return StringStringMapProto.newBuilder().setKey(key).setValue(
-                (env.get(key))).build();
+                (environment.get(key))).build();
           }
           
           @Override
@@ -424,7 +421,7 @@ implements ContainerLaunchContext {
         };
       }
     };
-    builder.addAllEnv(iterable);
+    builder.addAllEnvironment(iterable);
   }
 
   private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {

+ 6 - 12
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -188,17 +188,11 @@ message AMResponseProto {
 ////////////////////////////////////////////////////////////////////////
 message ApplicationSubmissionContextProto {
   optional ApplicationIdProto application_id = 1;
-  optional string application_name = 2;
-  optional ResourceProto master_capability = 3;
-  repeated StringURLMapProto resources = 4;
-  repeated StringLocalResourceMapProto resources_todo = 5;
-  repeated string fs_tokens = 6;
-  optional bytes fs_tokens_todo = 7;
-  repeated StringStringMapProto environment = 8;
-  repeated string command = 9;
-  optional string queue = 10;
-  optional PriorityProto priority = 11;
-  optional string user = 12;
+  optional string application_name = 2 [default = "N/A"];
+  optional string user = 3; 
+  optional string queue = 4 [default = "default"];
+  optional PriorityProto priority = 5;
+  optional ContainerLaunchContextProto am_container_spec = 6;
 }
 
 message YarnClusterMetricsProto {
@@ -242,7 +236,7 @@ message ContainerLaunchContextProto {
   repeated StringLocalResourceMapProto localResources = 4;
   optional bytes container_tokens = 5;
   repeated StringBytesMapProto service_data = 6;
-  repeated StringStringMapProto env = 7;
+  repeated StringStringMapProto environment = 7;
   repeated string command = 8;
 }
 

+ 6 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -219,6 +219,12 @@ public class YarnConfiguration extends Configuration {
     RM_PREFIX + "max-completed-applications";
   public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
   
+  /** Default application name */
+  public static final String DEFAULT_APPLICATION_NAME = "N/A";
+
+  /** Default queue name */
+  public static final String DEFAULT_QUEUE_NAME = "default";
+  
   ////////////////////////////////
   // Node Manager Configs
   ////////////////////////////////

+ 45 - 14
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java

@@ -67,28 +67,59 @@ public class CompositeService extends AbstractService {
         Service service = serviceList.get(i);
         service.start();
       }
-    } catch(Throwable e) {
+      super.start();
+    } catch (Throwable e) {
       LOG.error("Error starting services " + getName(), e);
-      for (int j = i-1; j >= 0; j--) {
-        Service service = serviceList.get(j);
-        try {
-          service.stop();
-        } catch(Throwable t) {
-          LOG.info("Error stopping " + service.getName(), t);
-        }
-      }
+      // Note that the state of the failed service is still INITED and not
+      // STARTED. Even though the last service is not started completely, still
+      // call stop() on all services including failed service to make sure cleanup
+      // happens.
+      stop(i);
       throw new YarnException("Failed to Start " + getName(), e);
     }
-    super.start();
+
   }
 
   public synchronized void stop() {
-    //stop in reserve order of start
-    for (int i = serviceList.size() - 1; i >= 0; i--) {
-      Service service = serviceList.get(i);
-      service.stop();
+    if (serviceList.size() > 0) {
+      stop(serviceList.size() - 1);
     }
     super.stop();
   }
 
+  private synchronized void stop(int numOfServicesStarted) {
+    // stop in reserve order of start
+    for (int i = numOfServicesStarted; i >= 0; i--) {
+      Service service = serviceList.get(i);
+      try {
+        service.stop();
+      } catch (Throwable t) {
+        LOG.info("Error stopping " + service.getName(), t);
+      }
+    }
+  }
+
+  /**
+   * JVM Shutdown hook for CompositeService which will stop the give
+   * CompositeService gracefully in case of JVM shutdown.
+   */
+  public static class CompositeServiceShutdownHook extends Thread {
+
+    private CompositeService compositeService;
+
+    public CompositeServiceShutdownHook(CompositeService compositeService) {
+      this.compositeService = compositeService;
+    }
+
+    @Override
+    public void run() {
+      try {
+        // Stop the Composite Service
+        compositeService.stop();
+      } catch (Throwable t) {
+        LOG.info("Error stopping " + compositeService.getName(), t);
+      }
+    }
+  }
+  
 }

+ 9 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/Times.java

@@ -30,10 +30,18 @@ public class Times {
       };
 
   public static long elapsed(long started, long finished) {
+    return Times.elapsed(started, finished, true);
+  }
+
+  public static long elapsed(long started, long finished, boolean isRunning) {
     if (finished > 0) {
       return finished - started;
     }
-    return started > 0 ? System.currentTimeMillis() - started : 0;
+    if (isRunning) {
+      return started > 0 ? System.currentTimeMillis() - started : 0;
+    } else {
+      return -1;
+    }
   }
 
   public static String format(long ts) {

+ 247 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java

@@ -0,0 +1,247 @@
+/**
+ * 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.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.hadoop.yarn.service.Service.STATE;
+import org.junit.Test;
+
+public class TestCompositeService {
+
+  private static final int NUM_OF_SERVICES = 5;
+
+  private static final int FAILED_SERVICE_SEQ_NUMBER = 2;
+
+  @Test
+  public void testCallSequence() {
+    ServiceManager serviceManager = new ServiceManager("ServiceManager");
+
+    // Add services
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      CompositeServiceImpl service = new CompositeServiceImpl(i);
+      serviceManager.addTestService(service);
+    }
+
+    CompositeServiceImpl[] services = serviceManager.getServices().toArray(
+        new CompositeServiceImpl[0]);
+
+    assertEquals("Number of registered services ", NUM_OF_SERVICES,
+        services.length);
+
+    Configuration conf = new Configuration();
+    // Initialise the composite service
+    serviceManager.init(conf);
+
+    // Verify the init() call sequence numbers for every service
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      assertEquals("For " + services[i]
+          + " service, init() call sequence number should have been ", i,
+          services[i].getCallSequenceNumber());
+    }
+
+    // Reset the call sequence numbers
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      services[i].reset();
+    }
+
+    serviceManager.start();
+
+    // Verify the start() call sequence numbers for every service
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      assertEquals("For " + services[i]
+          + " service, start() call sequence number should have been ", i,
+          services[i].getCallSequenceNumber());
+    }
+
+    // Reset the call sequence numbers
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      services[i].reset();
+    }
+
+    serviceManager.stop();
+
+    // Verify the stop() call sequence numbers for every service
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      assertEquals("For " + services[i]
+          + " service, stop() call sequence number should have been ",
+          ((NUM_OF_SERVICES - 1) - i), services[i].getCallSequenceNumber());
+    }
+
+  }
+
+  @Test
+  public void testServiceStartup() {
+    ServiceManager serviceManager = new ServiceManager("ServiceManager");
+
+    // Add services
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      CompositeServiceImpl service = new CompositeServiceImpl(i);
+      if (i == FAILED_SERVICE_SEQ_NUMBER) {
+        service.setThrowExceptionOnStart(true);
+      }
+      serviceManager.addTestService(service);
+    }
+
+    CompositeServiceImpl[] services = serviceManager.getServices().toArray(
+        new CompositeServiceImpl[0]);
+
+    Configuration conf = new Configuration();
+
+    // Initialise the composite service
+    serviceManager.init(conf);
+
+    // Start the composite service
+    try {
+      serviceManager.start();
+      fail("Exception should have been thrown due to startup failure of last service");
+    } catch (YarnException e) {
+      for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
+        if (i >= FAILED_SERVICE_SEQ_NUMBER) {
+          // Failed service state should be INITED
+          assertEquals("Service state should have been ", STATE.INITED,
+              services[NUM_OF_SERVICES - 1].getServiceState());
+        } else {
+          assertEquals("Service state should have been ", STATE.STOPPED,
+              services[i].getServiceState());
+        }
+      }
+
+    }
+  }
+
+  @Test
+  public void testServiceStop() {
+    ServiceManager serviceManager = new ServiceManager("ServiceManager");
+
+    // Add services
+    for (int i = 0; i < NUM_OF_SERVICES; i++) {
+      CompositeServiceImpl service = new CompositeServiceImpl(i);
+      if (i == FAILED_SERVICE_SEQ_NUMBER) {
+        service.setThrowExceptionOnStop(true);
+      }
+      serviceManager.addTestService(service);
+    }
+
+    CompositeServiceImpl[] services = serviceManager.getServices().toArray(
+        new CompositeServiceImpl[0]);
+
+    Configuration conf = new Configuration();
+
+    // Initialise the composite service
+    serviceManager.init(conf);
+
+    serviceManager.start();
+
+    // Start the composite service
+    try {
+      serviceManager.stop();
+    } catch (YarnException e) {
+      for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
+        assertEquals("Service state should have been ", STATE.STOPPED,
+            services[NUM_OF_SERVICES].getServiceState());
+      }
+    }
+  }
+
+  public static class CompositeServiceImpl extends CompositeService {
+
+    private static int counter = -1;
+
+    private int callSequenceNumber = -1;
+
+    private boolean throwExceptionOnStart;
+
+    private boolean throwExceptionOnStop;
+
+    public CompositeServiceImpl(int sequenceNumber) {
+      super(Integer.toString(sequenceNumber));
+    }
+
+    @Override
+    public synchronized void init(Configuration conf) {
+      counter++;
+      callSequenceNumber = counter;
+      super.init(conf);
+    }
+
+    @Override
+    public synchronized void start() {
+      if (throwExceptionOnStart) {
+        throw new YarnException("Fake service start exception");
+      }
+      counter++;
+      callSequenceNumber = counter;
+      super.start();
+    }
+
+    @Override
+    public synchronized void stop() {
+      counter++;
+      callSequenceNumber = counter;
+      if (throwExceptionOnStop) {
+        throw new YarnException("Fake service stop exception");
+      }
+      super.stop();
+    }
+
+    public static int getCounter() {
+      return counter;
+    }
+
+    public int getCallSequenceNumber() {
+      return callSequenceNumber;
+    }
+
+    public void reset() {
+      callSequenceNumber = -1;
+      counter = -1;
+    }
+
+    public void setThrowExceptionOnStart(boolean throwExceptionOnStart) {
+      this.throwExceptionOnStart = throwExceptionOnStart;
+    }
+
+    public void setThrowExceptionOnStop(boolean throwExceptionOnStop) {
+      this.throwExceptionOnStop = throwExceptionOnStop;
+    }
+
+    @Override
+    public String toString() {
+      return "Service " + getName();
+    }
+
+  }
+
+  public static class ServiceManager extends CompositeService {
+
+    public void addTestService(CompositeService service) {
+      addService(service);
+    }
+
+    public ServiceManager(String name) {
+      super(name);
+    }
+  }
+
+}

+ 15 - 5
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/test/WebAppTests.java

@@ -24,6 +24,8 @@ import org.apache.hadoop.yarn.webapp.View;
 import org.apache.hadoop.yarn.webapp.WebAppException;
 
 import java.lang.reflect.Method;
+import java.util.Map;
+
 import com.google.inject.Module;
 import com.google.inject.Scopes;
 import com.google.inject.servlet.RequestScoped;
@@ -126,22 +128,31 @@ public class WebAppTests {
     }
   }
 
-  @SuppressWarnings("unchecked")
   public static <T> Injector testController(Class<? extends Controller> ctrlr,
       String methodName) {
     return testController(ctrlr, methodName, null, null);
   }
 
   public static <T> Injector testPage(Class<? extends View> page, Class<T> api,
-                                      T impl, Module... modules) {
+                                      T impl, Map<String,String> params, Module... modules) {
     Injector injector = createMockInjector(api, impl, modules);
-    injector.getInstance(page).render();
+    View view = injector.getInstance(page);
+    if(params != null) {
+      for(Map.Entry<String, String> entry: params.entrySet()) {
+        view.set(entry.getKey(), entry.getValue());
+      }
+    }
+    view.render();
     flushOutput(injector);
     return injector;
   }
+  
+  public static <T> Injector testPage(Class<? extends View> page, Class<T> api,
+                                      T impl, Module... modules) {
+    return testPage(page, api, impl, null, modules);
+  }
 
   // convenience
-  @SuppressWarnings("unchecked")
   public static <T> Injector testPage(Class<? extends View> page) {
     return testPage(page, null, null);
   }
@@ -155,7 +166,6 @@ public class WebAppTests {
   }
 
   // convenience
-  @SuppressWarnings("unchecked")
   public static <T> Injector testBlock(Class<? extends SubView> block) {
     return testBlock(block, null, null);
   }

+ 11 - 11
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java

@@ -133,13 +133,6 @@ public class NodeManager extends CompositeService {
     dispatcher.register(ContainerManagerEventType.class, containerManager);
     addService(dispatcher);
 
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-          @Override
-          public void run() {
-            NodeManager.this.stop();
-          }
-        });
-
     DefaultMetricsSystem.initialize("NodeManager");
 
     // StatusUpdater should be added last so that it get started last 
@@ -200,10 +193,17 @@ public class NodeManager extends CompositeService {
 
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
-    NodeManager nodeManager = new NodeManager();
-    YarnConfiguration conf = new YarnConfiguration();
-    nodeManager.init(conf);
-    nodeManager.start();
+    try {
+      NodeManager nodeManager = new NodeManager();
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(nodeManager));
+      YarnConfiguration conf = new YarnConfiguration();
+      nodeManager.init(conf);
+      nodeManager.start();
+    } catch (Throwable t) {
+      LOG.fatal("Error starting NodeManager", t);
+      System.exit(-1);
+    }
   }
 
 }

+ 3 - 3
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -89,7 +89,7 @@ public class ContainerLaunch implements Callable<Integer> {
     final Map<Path,String> localResources = container.getLocalizedResources();
     String containerIdStr = ConverterUtils.toString(container.getContainerID());
     final String user = launchContext.getUser();
-    final Map<String,String> env = launchContext.getEnv();
+    final Map<String,String> env = launchContext.getEnvironment();
     final List<String> command = launchContext.getCommands();
     int ret = -1;
 
@@ -109,7 +109,7 @@ public class ContainerLaunch implements Callable<Integer> {
       }
       launchContext.setCommands(newCmds);
 
-      Map<String, String> envs = launchContext.getEnv();
+      Map<String, String> envs = launchContext.getEnvironment();
       Map<String, String> newEnvs = new HashMap<String, String>(envs.size());
       for (Entry<String, String> entry : envs.entrySet()) {
         newEnvs.put(
@@ -118,7 +118,7 @@ public class ContainerLaunch implements Callable<Integer> {
                 ApplicationConstants.LOG_DIR_EXPANSION_VAR,
                 containerLogDir.toUri().getPath()));
       }
-      launchContext.setEnv(newEnvs);
+      launchContext.setEnvironment(newEnvs);
       // /////////////////////////// End of variable expansion
 
       FileContext lfs = FileContext.getLocalFSFileContext();

+ 16 - 8
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java

@@ -107,9 +107,10 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.even
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerSecurityInfo;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
-public class ResourceLocalizationService extends AbstractService
+public class ResourceLocalizationService extends CompositeService
     implements EventHandler<LocalizationEvent>, LocalizationProtocol {
 
   private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class);
@@ -201,9 +202,8 @@ public class ResourceLocalizationService extends AbstractService
     localizationServerAddress = NetUtils.createSocketAddr(
       conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS));
     localizerTracker = createLocalizerTracker(conf);
+    addService(localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
-    cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
-        cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
     super.init(conf);
   }
 
@@ -214,6 +214,8 @@ public class ResourceLocalizationService extends AbstractService
 
   @Override
   public void start() {
+    cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
+        cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
     server = createServer();
     LOG.info("Localizer started on port " + server.getPort());
     server.start();
@@ -247,9 +249,7 @@ public class ResourceLocalizationService extends AbstractService
     if (server != null) {
       server.close();
     }
-    if (localizerTracker != null) {
-      localizerTracker.stop();
-    }
+    cacheCleanup.shutdown();
     super.stop();
   }
 
@@ -403,7 +403,7 @@ public class ResourceLocalizationService extends AbstractService
   /**
    * Sub-component handling the spawning of {@link ContainerLocalizer}s
    */
-  class LocalizerTracker implements EventHandler<LocalizerEvent> {
+  class LocalizerTracker extends AbstractService implements EventHandler<LocalizerEvent>  {
 
     private final PublicLocalizer publicLocalizer;
     private final Map<String,LocalizerRunner> privLocalizers;
@@ -414,9 +414,15 @@ public class ResourceLocalizationService extends AbstractService
 
     LocalizerTracker(Configuration conf,
         Map<String,LocalizerRunner> privLocalizers) {
+      super(LocalizerTracker.class.getName());
       this.publicLocalizer = new PublicLocalizer(conf);
       this.privLocalizers = privLocalizers;
+    }
+    
+    @Override
+    public synchronized void start() {
       publicLocalizer.start();
+      super.start();
     }
 
     public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
@@ -435,12 +441,14 @@ public class ResourceLocalizationService extends AbstractService
         return localizer.update(status.getResources());
       }
     }
-
+    
+    @Override
     public void stop() {
       for (LocalizerRunner localizer : privLocalizers.values()) {
         localizer.interrupt();
       }
       publicLocalizer.interrupt();
+      super.stop();
     }
 
     @Override

+ 7 - 6
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -71,7 +71,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstant
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.service.AbstractService;
@@ -90,7 +89,6 @@ public class ClientRMService extends AbstractService implements
   final private AtomicInteger applicationCounter = new AtomicInteger(0);
   final private YarnScheduler scheduler;
   final private RMContext rmContext;
-  private final AMLivelinessMonitor amLivelinessMonitor;
   private final RMAppManager rmAppManager;
 
   private String clientServiceBindAddress;
@@ -106,7 +104,6 @@ public class ClientRMService extends AbstractService implements
     super(ClientRMService.class.getName());
     this.scheduler = scheduler;
     this.rmContext = rmContext;
-    this.amLivelinessMonitor = rmContext.getAMLivelinessMonitor();
     this.rmAppManager = rmAppManager;
   }
   
@@ -195,15 +192,18 @@ public class ClientRMService extends AbstractService implements
       SubmitApplicationRequest request) throws YarnRemoteException {
     ApplicationSubmissionContext submissionContext = request
         .getApplicationSubmissionContext();
-    ApplicationId applicationId = null;
-    String user = null;
+    ApplicationId applicationId = submissionContext.getApplicationId();
+    String user = submissionContext.getUser();
     try {
       user = UserGroupInformation.getCurrentUser().getShortUserName();
-      applicationId = submissionContext.getApplicationId();
       if (rmContext.getRMApps().get(applicationId) != null) {
         throw new IOException("Application with id " + applicationId
             + " is already present! Cannot add a duplicate!");
       }
+      
+      // Safety 
+      submissionContext.setUser(user);
+      
       // This needs to be synchronous as the client can query 
       // immediately following the submission to get the application status.
       // So call handle directly and do not send an event.
@@ -226,6 +226,7 @@ public class ClientRMService extends AbstractService implements
     return response;
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public FinishApplicationResponse finishApplication(
       FinishApplicationRequest request) throws YarnRemoteException {

+ 23 - 11
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java

@@ -210,7 +210,9 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
     }
   }
 
-  protected synchronized void submitApplication(ApplicationSubmissionContext submissionContext) {
+  @SuppressWarnings("unchecked")
+  protected synchronized void submitApplication(
+      ApplicationSubmissionContext submissionContext) {
     ApplicationId applicationId = submissionContext.getApplicationId();
     RMApp application = null;
     try {
@@ -224,27 +226,37 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
         clientTokenStr = clientToken.encodeToUrlString();
         LOG.debug("Sending client token as " + clientTokenStr);
       }
-      submissionContext.setQueue(submissionContext.getQueue() == null
-          ? "default" : submissionContext.getQueue());
-      submissionContext.setApplicationName(submissionContext
-          .getApplicationName() == null ? "N/A" : submissionContext
-          .getApplicationName());
+      
+      // Sanity checks
+      if (submissionContext.getQueue() == null) {
+        submissionContext.setQueue(YarnConfiguration.DEFAULT_QUEUE_NAME);
+      }
+      if (submissionContext.getApplicationName() == null) {
+        submissionContext.setApplicationName(
+            YarnConfiguration.DEFAULT_APPLICATION_NAME);
+      }
+
+      // Store application for recovery
       ApplicationStore appStore = rmContext.getApplicationsStore()
           .createApplicationStore(submissionContext.getApplicationId(),
           submissionContext);
+      
+      // Create RMApp
       application = new RMAppImpl(applicationId, rmContext,
           this.conf, submissionContext.getApplicationName(), user,
           submissionContext.getQueue(), submissionContext, clientTokenStr,
-          appStore, rmContext.getAMLivelinessMonitor(), this.scheduler,
+          appStore, this.scheduler,
           this.masterService);
 
-      if (rmContext.getRMApps().putIfAbsent(applicationId, application) != null) {
+      if (rmContext.getRMApps().putIfAbsent(applicationId, application) != 
+          null) {
         LOG.info("Application with id " + applicationId + 
             " is already present! Cannot add a duplicate!");
-        // don't send event through dispatcher as it will be handled by app already
-        // present with this id.
+        // don't send event through dispatcher as it will be handled by app 
+        // already present with this id.
         application.handle(new RMAppRejectedEvent(applicationId,
-            "Application with this id is already present! Cannot add a duplicate!"));
+            "Application with this id is already present! " +
+            "Cannot add a duplicate!"));
       } else {
         this.rmContext.getDispatcher().getEventHandler().handle(
             new RMAppEvent(applicationId, RMAppEventType.START));

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

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 
 public class RMAppManagerSubmitEvent extends RMAppManagerEvent {

+ 7 - 10
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java

@@ -22,7 +22,6 @@ package org.apache.hadoop.yarn.server.resourcemanager;
 import java.io.IOException;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -45,8 +44,8 @@ import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
+import org.apache.hadoop.yarn.server.resourcemanager.recovery.Store.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -101,7 +100,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
   private SchedulerEventDispatcher schedulerDispatcher;
   protected RMAppManager rmAppManager;
 
-  private final AtomicBoolean shutdown = new AtomicBoolean(false);
   private WebApp webApp;
   private RMContext rmContext;
   private final Store store;
@@ -490,20 +488,19 @@ public class ResourceManager extends CompositeService implements Recoverable {
   
   public static void main(String argv[]) {
     StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
-    ResourceManager resourceManager = null;
     try {
       Configuration conf = new YarnConfiguration();
       Store store =  StoreFactory.getStore(conf);
-      resourceManager = new ResourceManager(store);
+      ResourceManager resourceManager = new ResourceManager(store);
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(resourceManager));
       resourceManager.init(conf);
       //resourceManager.recover(store.restore());
       //store.doneWithRecovery();
       resourceManager.start();
-    } catch (Throwable e) {
-      LOG.error("Error starting RM", e);
-      if (resourceManager != null) {
-        resourceManager.stop();
-      }
+    } catch (Throwable t) {
+      LOG.fatal("Error starting ResourceManager", t);
+      System.exit(-1);
     }
   }
 }

+ 23 - 25
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -120,7 +119,8 @@ public class AMLauncher implements Runnable {
         + " for AM " + application.getAppAttemptId());  
     ContainerLaunchContext launchContext =
         createAMContainerLaunchContext(applicationContext, masterContainerID);
-    StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
+    StartContainerRequest request = 
+        recordFactory.newRecordInstance(StartContainerRequest.class);
     request.setContainerLaunchContext(launchContext);
     containerMgrProxy.startContainer(request);
     LOG.info("Done launching container " + application.getMasterContainer() 
@@ -130,7 +130,8 @@ public class AMLauncher implements Runnable {
   private void cleanup() throws IOException {
     connect();
     ContainerId containerId = application.getMasterContainer().getId();
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
+    StopContainerRequest stopRequest = 
+        recordFactory.newRecordInstance(StopContainerRequest.class);
     stopRequest.setContainerId(containerId);
     containerMgrProxy.stopContainer(stopRequest);
   }
@@ -145,7 +146,7 @@ public class AMLauncher implements Runnable {
     final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
 
     UserGroupInformation currentUser =
-        UserGroupInformation.createRemoteUser("TODO"); // TODO
+        UserGroupInformation.createRemoteUser("yarn"); // TODO
     if (UserGroupInformation.isSecurityEnabled()) {
       ContainerToken containerToken = container.getContainerToken();
       Token<ContainerTokenIdentifier> token =
@@ -170,8 +171,8 @@ public class AMLauncher implements Runnable {
       ContainerId containerID) throws IOException {
 
     // Construct the actual Container
-    ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    container.setCommands(applicationMasterContext.getCommandList());
+    ContainerLaunchContext container = 
+        applicationMasterContext.getAMContainerSpec();
     StringBuilder mergedCommand = new StringBuilder();
     String failCount = Integer.toString(application.getAppAttemptId()
         .getAttemptId());
@@ -189,34 +190,28 @@ public class AMLauncher implements Runnable {
    
     LOG.info("Command to launch container " + 
         containerID + " : " + mergedCommand);
-    Map<String, String> environment = 
-        applicationMasterContext.getAllEnvironment();
-    environment.putAll(setupTokensInEnv(applicationMasterContext));
-    container.setEnv(environment);
-
-    // Construct the actual Container
+    
+    // Finalize the container
     container.setContainerId(containerID);
     container.setUser(applicationMasterContext.getUser());
-    container.setResource(applicationMasterContext.getMasterCapability());
-    container.setLocalResources(applicationMasterContext.getAllResourcesTodo());
-    container.setContainerTokens(applicationMasterContext.getFsTokensTodo());
+    setupTokensAndEnv(container);
+    
     return container;
   }
 
-  private Map<String, String> setupTokensInEnv(
-      ApplicationSubmissionContext asc)
+  private void setupTokensAndEnv(
+      ContainerLaunchContext container)
       throws IOException {
-    Map<String, String> env =
-      new HashMap<String, String>();
+    Map<String, String> environment = container.getEnvironment();
     if (UserGroupInformation.isSecurityEnabled()) {
       // TODO: Security enabled/disabled info should come from RM.
 
       Credentials credentials = new Credentials();
 
       DataInputByteBuffer dibb = new DataInputByteBuffer();
-      if (asc.getFsTokensTodo() != null) {
+      if (container.getContainerTokens() != null) {
         // TODO: Don't do this kind of checks everywhere.
-        dibb.reset(asc.getFsTokensTodo());
+        dibb.reset(container.getContainerTokens());
         credentials.readTokenStorageStream(dibb);
       }
 
@@ -236,14 +231,16 @@ public class AMLauncher implements Runnable {
       token.setService(new Text(resolvedAddr));
       String appMasterTokenEncoded = token.encodeToUrlString();
       LOG.debug("Putting appMaster token in env : " + appMasterTokenEncoded);
-      env.put(ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
+      environment.put(
+          ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
           appMasterTokenEncoded);
 
       // Add the RM token
       credentials.addToken(new Text(resolvedAddr), token);
       DataOutputBuffer dob = new DataOutputBuffer();
       credentials.writeTokenStorageToStream(dob);
-      asc.setFsTokensTodo(ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
+      container.setContainerTokens(
+          ByteBuffer.wrap(dob.getData(), 0, dob.getLength()));
 
       ApplicationTokenIdentifier identifier = new ApplicationTokenIdentifier(
           application.getAppAttemptId().getApplicationId());
@@ -252,9 +249,10 @@ public class AMLauncher implements Runnable {
       String encoded =
           Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
       LOG.debug("The encoded client secret-key to be put in env : " + encoded);
-      env.put(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME, encoded);
+      environment.put(
+          ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME, 
+          encoded);
     }
-    return env;
   }
   
   @SuppressWarnings("unchecked")

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

@@ -86,7 +86,6 @@ public class RMAppImpl implements RMApp {
   // Mutable fields
   private long startTime;
   private long finishTime;
-  private AMLivelinessMonitor amLivelinessMonitor;
   private RMAppAttempt currentAttempt;
 
   private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
@@ -163,7 +162,7 @@ public class RMAppImpl implements RMApp {
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
       Configuration config, String name, String user, String queue,
       ApplicationSubmissionContext submissionContext, String clientTokenStr,
-      ApplicationStore appStore, AMLivelinessMonitor amLivelinessMonitor,
+      ApplicationStore appStore, 
       YarnScheduler scheduler, ApplicationMasterService masterService) {
 
     this.applicationId = applicationId;
@@ -176,7 +175,6 @@ public class RMAppImpl implements RMApp {
     this.submissionContext = submissionContext;
     this.clientTokenStr = clientTokenStr;
     this.appStore = appStore;
-    this.amLivelinessMonitor = amLivelinessMonitor;
     this.scheduler = scheduler;
     this.masterService = masterService;
     this.startTime = System.currentTimeMillis();
@@ -380,6 +378,7 @@ public class RMAppImpl implements RMApp {
     }
   }
 
+  @SuppressWarnings("unchecked")
   private void createNewAttempt() {
     ApplicationAttemptId appAttemptId = Records
         .newRecord(ApplicationAttemptId.class);
@@ -434,6 +433,7 @@ public class RMAppImpl implements RMApp {
       return nodes;
     }
 
+    @SuppressWarnings("unchecked")
     public void transition(RMAppImpl app, RMAppEvent event) {
       Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
       for (NodeId nodeId : nodes) {

+ 2 - 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

@@ -84,6 +84,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
                              RMAppAttemptEvent> stateMachine;
 
   private final RMContext rmContext;
+  @SuppressWarnings("rawtypes")
   private final EventHandler eventHandler;
   private final YarnScheduler scheduler;
   private final ApplicationMasterService masterService;
@@ -459,7 +460,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
       // Request a container for the AM.
       ResourceRequest request = BuilderUtils.newResourceRequest(
           AM_CONTAINER_PRIORITY, "*", appAttempt.submissionContext
-              .getMasterCapability(), 1);
+              .getAMContainerSpec().getResource(), 1);
       LOG.debug("About to request resources for AM of "
           + appAttempt.applicationAttemptId + " required " + request);
 

+ 6 - 10
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java

@@ -1023,21 +1023,17 @@ public class LeafQueue implements CSQueue {
     // Check if we need containers on this rack 
     ResourceRequest rackLocalRequest = 
       application.getResourceRequest(priority, node.getRackName());
+    if (rackLocalRequest == null || rackLocalRequest.getNumContainers() <= 0) {
+      return false;
+    }
+      
+    // If we are here, we do need containers on this rack for RACK_LOCAL req
     if (type == NodeType.RACK_LOCAL) {
-      if (rackLocalRequest == null) {
-        return false;
-      } else {
-        return rackLocalRequest.getNumContainers() > 0;      
-      }
+      return true;
     }
 
     // Check if we need containers on this host
     if (type == NodeType.NODE_LOCAL) {
-      // First: Do we need containers on this rack?
-      if (rackLocalRequest != null && rackLocalRequest.getNumContainers() == 0) {
-        return false;
-      }
-      
       // Now check if we need containers on this host...
       ResourceRequest nodeLocalRequest = 
         application.getResourceRequest(priority, node.getHostName());

+ 16 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java

@@ -289,6 +289,7 @@ public class FifoScheduler implements ResourceScheduler {
     return nodes.get(nodeId);
   }
   
+  @SuppressWarnings("unchecked")
   private synchronized void addApplication(ApplicationAttemptId appAttemptId,
       String queueName, String user) {
     // TODO: Fix store
@@ -440,6 +441,14 @@ public class FifoScheduler implements ResourceScheduler {
     ResourceRequest request = 
       application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
     if (request != null) {
+      // Don't allocate on this node if we don't need containers on this rack
+      ResourceRequest rackRequest =
+          application.getResourceRequest(priority, 
+              node.getRMNode().getRackName());
+      if (rackRequest == null || rackRequest.getNumContainers() <= 0) {
+        return 0;
+      }
+      
       int assignableContainers = 
         Math.min(
             getMaxAllocatableContainers(application, priority, node, 
@@ -458,6 +467,13 @@ public class FifoScheduler implements ResourceScheduler {
     ResourceRequest request = 
       application.getResourceRequest(priority, node.getRMNode().getRackName());
     if (request != null) {
+      // Don't allocate on this rack if the application doens't need containers
+      ResourceRequest offSwitchRequest =
+          application.getResourceRequest(priority, SchedulerNode.ANY);
+      if (offSwitchRequest.getNumContainers() <= 0) {
+        return 0;
+      }
+      
       int assignableContainers = 
         Math.min(
             getMaxAllocatableContainers(application, priority, node, 

+ 2 - 2
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,7 +23,6 @@ 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.util.Apps;
 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;
@@ -59,7 +58,8 @@ class AppsBlock extends HtmlBlock {
       String appId = app.getApplicationId().toString();
       String trackingUrl = app.getTrackingUrl();
       String ui = trackingUrl == null || trackingUrl.isEmpty() ? "UNASSIGNED" :
-          (app.getFinishTime() == 0 ? "ApplicationMaster" : "JobHistory");
+          (app.getFinishTime() == 0 ? 
+              "ApplicationMaster URL" : "JobHistory URL");
       String percent = String.format("%.1f", app.getProgress() * 100);
       tbody.
         tr().

+ 7 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -29,6 +29,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
@@ -81,13 +82,17 @@ public class MockRM extends ResourceManager {
     ApplicationId appId = resp.getApplicationId();
     
     SubmitApplicationRequest req = Records.newRecord(SubmitApplicationRequest.class);
-    ApplicationSubmissionContext sub = Records.newRecord(ApplicationSubmissionContext.class);
+    ApplicationSubmissionContext sub = 
+        Records.newRecord(ApplicationSubmissionContext.class);
     sub.setApplicationId(appId);
     sub.setApplicationName("");
     sub.setUser("");
+    ContainerLaunchContext clc = 
+        Records.newRecord(ContainerLaunchContext.class);
     Resource capability = Records.newRecord(Resource.class);
     capability.setMemory(masterMemory);
-    sub.setMasterCapability(capability);
+    clc.setResource(capability);
+    sub.setAMContainerSpec(clc);
     req.setApplicationSubmissionContext(sub);
     
     client.submitApplication(req);

+ 15 - 25
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -18,19 +18,12 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager;
 
-import static org.mockito.Mockito.*;
 
-import java.util.ArrayList;
 import java.util.List;
-import java.util.LinkedList;
-import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 
-
 import junit.framework.Assert;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -44,7 +37,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager;
@@ -63,8 +55,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.service.Service;
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Lists;
@@ -75,7 +65,6 @@ import com.google.common.collect.Lists;
  */
 
 public class TestAppManager{
-  private static final Log LOG = LogFactory.getLog(TestAppManager.class);
   private static RMAppEventType appEventType = RMAppEventType.KILL; 
 
   public synchronized RMAppEventType getAppEventType() {
@@ -117,10 +106,8 @@ public class TestAppManager{
   public class TestAppManagerDispatcher implements
       EventHandler<RMAppManagerEvent> {
 
-    private final RMContext rmContext;
 
-    public TestAppManagerDispatcher(RMContext rmContext) {
-      this.rmContext = rmContext;
+    public TestAppManagerDispatcher() {
     }
 
     @Override
@@ -132,15 +119,11 @@ public class TestAppManager{
   public class TestDispatcher implements
       EventHandler<RMAppEvent> {
 
-    private final RMContext rmContext;
-
-    public TestDispatcher(RMContext rmContext) {
-      this.rmContext = rmContext;
+    public TestDispatcher() {
     }
 
     @Override
     public void handle(RMAppEvent event) {
-      ApplicationId appID = event.getApplicationId();
       //RMApp rmApp = this.rmContext.getRMApps().get(appID);
       setAppEventType(event.getType());
       System.out.println("in handle routine " + getAppEventType().toString());
@@ -178,7 +161,8 @@ public class TestAppManager{
     public void setCompletedAppsMax(int max) {
       super.setCompletedAppsMax(max);
     }
-    public void submitApplication(ApplicationSubmissionContext submissionContext) {
+    public void submitApplication(
+        ApplicationSubmissionContext submissionContext) {
       super.submitApplication(submissionContext);
     }
   }
@@ -336,8 +320,9 @@ public class TestAppManager{
   }
 
   protected void setupDispatcher(RMContext rmContext, Configuration conf) {
-    TestDispatcher testDispatcher = new TestDispatcher(rmContext);
-    TestAppManagerDispatcher testAppManagerDispatcher = new TestAppManagerDispatcher(rmContext);
+    TestDispatcher testDispatcher = new TestDispatcher();
+    TestAppManagerDispatcher testAppManagerDispatcher = 
+        new TestAppManagerDispatcher();
     rmContext.getDispatcher().register(RMAppEventType.class, testDispatcher);
     rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher);
     ((Service)rmContext.getDispatcher()).init(conf);
@@ -359,7 +344,8 @@ public class TestAppManager{
 
     ApplicationId appID = MockApps.newAppID(1);
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    ApplicationSubmissionContext context = 
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(appID);
     setupDispatcher(rmContext, conf);
 
@@ -367,8 +353,12 @@ public class TestAppManager{
     RMApp app = rmContext.getRMApps().get(appID);
     Assert.assertNotNull("app is null", app);
     Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
-    Assert.assertEquals("app name doesn't match", "N/A", app.getName());
-    Assert.assertEquals("app queue doesn't match", "default", app.getQueue());
+    Assert.assertEquals("app name doesn't match", 
+        YarnConfiguration.DEFAULT_APPLICATION_NAME, 
+        app.getName());
+    Assert.assertEquals("app queue doesn't match", 
+        YarnConfiguration.DEFAULT_QUEUE_NAME, 
+        app.getQueue());
     Assert.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
     Assert.assertNotNull("app store is null", app.getApplicationStore());
 

+ 1 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/TestRMAppTransitions.java

@@ -128,7 +128,7 @@ public class TestRMAppTransitions {
     RMApp application = new RMAppImpl(applicationId, rmContext,
           conf, name, user,
           queue, submissionContext, clientTokenStr,
-          appStore, rmContext.getAMLivelinessMonitor(), scheduler,
+          appStore, scheduler,
           masterService);
 
     testAppStartState(applicationId, user, name, queue, application);

+ 101 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.java

@@ -625,7 +625,6 @@ public class TestLeafQueue {
   }
   
   
-  
   @Test
   public void testLocalityScheduling() throws Exception {
 
@@ -876,6 +875,107 @@ public class TestLeafQueue {
 
   }
   
+  @Test
+  public void testSchedulingConstraints() throws Exception {
+
+    // Manipulate queue 'a'
+    LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
+
+    // User
+    String user_0 = "user_0";
+    
+    // Submit applications
+    final ApplicationAttemptId appAttemptId_0 = 
+        TestUtils.getMockApplicationAttemptId(0, 0); 
+    SchedulerApp app_0 = 
+        spy(new SchedulerApp(appAttemptId_0, user_0, a, rmContext, null));
+    a.submitApplication(app_0, user_0, A);
+    
+    // Setup some nodes and racks
+    String host_0_0 = "host_0_0";
+    String rack_0 = "rack_0";
+    SchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 8*GB);
+    String host_0_1 = "host_0_1";
+    SchedulerNode node_0_1 = TestUtils.getMockNode(host_0_1, rack_0, 0, 8*GB);
+    
+    
+    String host_1_0 = "host_1_0";
+    String rack_1 = "rack_1";
+    SchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB);
+    
+    final int numNodes = 3;
+    Resource clusterResource = Resources.createResource(numNodes * (8*GB));
+    when(csContext.getNumClusterNodes()).thenReturn(numNodes);
+
+    // Setup resource-requests and submit
+    Priority priority = TestUtils.createMockPriority(1);
+    List<ResourceRequest> app_0_requests_0 = new ArrayList<ResourceRequest>();
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(host_0_0, 1*GB, 1, 
+            priority, recordFactory));
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(host_0_1, 1*GB, 1, 
+            priority, recordFactory));
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(rack_0, 1*GB, 1, 
+            priority, recordFactory));
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(host_1_0, 1*GB, 1, 
+            priority, recordFactory));
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(rack_1, 1*GB, 1, 
+            priority, recordFactory));
+    app_0.updateResourceRequests(app_0_requests_0);
+
+    // Start testing...
+    
+    // Add one request
+    app_0_requests_0.clear();
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 1, // only one 
+            priority, recordFactory));
+    app_0.updateResourceRequests(app_0_requests_0);
+    
+    // NODE_LOCAL - node_0_1
+    a.assignContainers(clusterResource, node_0_0);
+    verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_0_0), 
+        any(Priority.class), any(ResourceRequest.class), any(Container.class));
+    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
+    assertEquals(0, app_0.getTotalRequiredResources(priority));
+
+    // No allocation on node_1_0 even though it's node/rack local since
+    // required(ANY) == 0
+    a.assignContainers(clusterResource, node_1_0);
+    verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
+        any(Priority.class), any(ResourceRequest.class), any(Container.class));
+    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // Still zero
+                                                               // since #req=0
+    assertEquals(0, app_0.getTotalRequiredResources(priority));
+    
+    // Add one request
+    app_0_requests_0.clear();
+    app_0_requests_0.add(
+        TestUtils.createResourceRequest(RMNodeImpl.ANY, 1*GB, 1, // only one 
+            priority, recordFactory));
+    app_0.updateResourceRequests(app_0_requests_0);
+
+    // No allocation on node_0_1 even though it's node/rack local since
+    // required(rack_1) == 0
+    a.assignContainers(clusterResource, node_0_1);
+    verify(app_0, never()).allocate(any(NodeType.class), eq(node_1_0), 
+        any(Priority.class), any(ResourceRequest.class), any(Container.class));
+    assertEquals(1, app_0.getSchedulingOpportunities(priority)); 
+    assertEquals(1, app_0.getTotalRequiredResources(priority));
+    
+    // NODE_LOCAL - node_1
+    a.assignContainers(clusterResource, node_1_0);
+    verify(app_0).allocate(eq(NodeType.NODE_LOCAL), eq(node_1_0), 
+        any(Priority.class), any(ResourceRequest.class), any(Container.class));
+    assertEquals(0, app_0.getSchedulingOpportunities(priority)); // should reset
+    assertEquals(0, app_0.getTotalRequiredResources(priority));
+
+  }
+  
   @After
   public void tearDown() throws Exception {
   }

+ 10 - 10
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerTokenSecretManager.java

@@ -27,6 +27,8 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import junit.framework.Assert;
@@ -54,10 +56,10 @@ import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterReque
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationMaster;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -77,6 +79,7 @@ import org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -137,15 +140,11 @@ public class TestContainerTokenSecretManager {
     ApplicationSubmissionContext appSubmissionContext =
         recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     appSubmissionContext.setApplicationId(appID);
-    appSubmissionContext.setMasterCapability(recordFactory
-        .newRecordInstance(Resource.class));
-    appSubmissionContext.getMasterCapability().setMemory(1024);
-//    appSubmissionContext.resources = new HashMap<String, URL>();
+    ContainerLaunchContext amContainer =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    amContainer.setResource(Resources.createResource(1024));
+    amContainer.setCommands(Arrays.asList("sleep", "100"));
     appSubmissionContext.setUser("testUser");
-//    appSubmissionContext.environment = new HashMap<String, String>();
-//    appSubmissionContext.command = new ArrayList<String>();
-    appSubmissionContext.addCommand("sleep");
-    appSubmissionContext.addCommand("100");
 
     // TODO: Use a resource to work around bugs. Today NM doesn't create local
     // app-dirs if there are no file to download!!
@@ -162,10 +161,11 @@ public class TestContainerTokenSecretManager {
     rsrc.setTimestamp(file.lastModified());
     rsrc.setType(LocalResourceType.FILE);
     rsrc.setVisibility(LocalResourceVisibility.PRIVATE);
-    appSubmissionContext.setResourceTodo("testFile", rsrc);
+    amContainer.setLocalResources(Collections.singletonMap("testFile", rsrc));
     SubmitApplicationRequest submitRequest = recordFactory
         .newRecordInstance(SubmitApplicationRequest.class);
     submitRequest.setApplicationSubmissionContext(appSubmissionContext);
+    appSubmissionContext.setAMContainerSpec(amContainer);
     resourceManager.getClientRMService().submitApplication(submitRequest);
 
     // Wait till container gets allocated for AM

+ 3 - 1
hadoop-mapreduce-project/src/webapps/job/queueinfo.jsp

@@ -21,6 +21,7 @@
 <%@ page
 contentType="text/html; charset=UTF-8"
 import="org.apache.hadoop.mapred.*"
+import="org.apache.commons.lang.StringUtils"
 import="javax.servlet.*"
 import="javax.servlet.http.*"
 import="java.io.*"
@@ -51,7 +52,8 @@ import="java.util.*"
       return "";
     }
     for (JobQueueInfo queueInfo : children) {
-      String variableName = queueInfo.getQueueName().replace(":", "_");
+      String variableName = StringUtils.replaceChars(queueInfo.getQueueName(),  
+                                 ":-*+#.^", "_______");
       String label = queueInfo.getQueueName().split(":")[queueInfo
           .getQueueName().split(":").length - 1];
       str.append(String.format(