Browse Source

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 years ago
parent
commit
6af0c27930
87 changed files with 2690 additions and 1650 deletions
  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-7607. Simplify the RPC proxy cleanup process. (atm)
 
 
+    HADOOP-7635. RetryInvocationHandler should release underlying resources on
+                 close (atm)
+
   BUGS
   BUGS
 
 
     HADOOP-7606. Upgrade Jackson to version 1.7.1 to match the version required
     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-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
 Release 0.23.0 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
@@ -583,6 +588,9 @@ Release 0.23.0 - Unreleased
 
 
     HADOOP-7626. Bugfix for a config generator (Eric Yang via ddas)
     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
 Release 0.22.0 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   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
   static {                                      // register a ctor
     WritableFactories.setFactory(FsPermission.class, FACTORY);
     WritableFactories.setFactory(FsPermission.class, FACTORY);
+    WritableFactories.setFactory(ImmutableFsPermission.class, FACTORY);
   }
   }
 
 
   /** Create an immutable {@link FsPermission} object. */
   /** Create an immutable {@link FsPermission} object. */
   public static FsPermission createImmutable(short permission) {
   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
   //POSIX permission style
@@ -301,4 +295,16 @@ public class FsPermission implements Writable {
 
 
     return new FsPermission((short)n);
     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;
 package org.apache.hadoop.io.retry;
 
 
+import java.io.IOException;
+
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.ipc.RPC;
 
 
 /**
 /**
  * An implementation of {@link FailoverProxyProvider} which does nothing in the
  * An implementation of {@link FailoverProxyProvider} which does nothing in the
@@ -49,4 +52,9 @@ public class DefaultFailoverProxyProvider implements FailoverProxyProvider {
     // Nothing to do.
     // 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;
 package org.apache.hadoop.io.retry;
 
 
+import java.io.Closeable;
+
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 
 
 /**
 /**
@@ -27,7 +29,7 @@ import org.apache.hadoop.classification.InterfaceStability;
  * {@link RetryPolicy}.
  * {@link RetryPolicy}.
  */
  */
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public interface FailoverProxyProvider {
+public interface FailoverProxyProvider extends Closeable {
 
 
   /**
   /**
    * Get the proxy object which should be used until the next failover event
    * 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;
 package org.apache.hadoop.io.retry;
 
 
+import java.io.Closeable;
+import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
@@ -27,7 +29,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.retry.RetryPolicy.RetryAction;
 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);
   public static final Log LOG = LogFactory.getLog(RetryInvocationHandler.class);
   private FailoverProxyProvider proxyProvider;
   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"?>
 <?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. -->
 <!-- This is the configuration file for the resource manager in Hadoop. -->
 <!-- You can configure various scheduling parameters related to queues. -->
 <!-- You can configure various scheduling parameters related to queues. -->
 <!-- The properties for a queue follow a naming convention,such as, -->
 <!-- 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
 #Logging Implementation
 
 
 #Log4J
 #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 version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <?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. -->
 <!-- Put site-specific property overrides in this file. -->
 
 
 <configuration>
 <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.
 # Set Hadoop-specific environment variables here.
 
 
 # The only required environment variable is JAVA_HOME.  All others are
 # 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 version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <?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. -->
 <!-- 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 version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <?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>
 <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 version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <?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>
 <configuration>
 <property>
 <property>
 <name>mapred.queue.default.acl-submit-job</name>
 <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 version="1.0"?>
 <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
 <?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. -->
 <!-- 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.cluster.local.dir=${HADOOP_MAPRED_DIR}
 mapreduce.tasktracker.group=${HADOOP_GROUP}
 mapreduce.tasktracker.group=${HADOOP_GROUP}
 hadoop.log.dir=${HADOOP_LOG_DIR}/${HADOOP_MR_USER}
 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() {
     public Class<?> getInterface() {
       return iface;
       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;
 package org.apache.hadoop.ipc;
 
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.io.IOException;
 import java.net.ConnectException;
 import java.net.ConnectException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ManagementFactory;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadInfo;
 import java.lang.management.ThreadMXBean;
 import java.lang.management.ThreadMXBean;
+import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
-import junit.framework.TestCase;
+import javax.net.SocketFactory;
 
 
 import org.apache.commons.logging.*;
 import org.apache.commons.logging.*;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.UTF8;
 import org.apache.hadoop.io.Writable;
 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.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.security.authorize.Service;
 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.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;
 import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
 import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
@@ -49,18 +59,22 @@ import static org.apache.hadoop.test.MetricsAsserts.*;
 import static org.mockito.Mockito.*;
 import static org.mockito.Mockito.*;
 
 
 /** Unit tests for RPC. */
 /** Unit tests for RPC. */
-public class TestRPC extends TestCase {
+@SuppressWarnings("deprecation")
+public class TestRPC {
   private static final String ADDRESS = "0.0.0.0";
   private static final String ADDRESS = "0.0.0.0";
 
 
   public static final Log LOG =
   public static final Log LOG =
     LogFactory.getLog(TestRPC.class);
     LogFactory.getLog(TestRPC.class);
   
   
   private static Configuration conf = new Configuration();
   private static Configuration conf = new Configuration();
+  
+  static {
+    conf.setClass("rpc.engine." + StoppedProtocol.class.getName(),
+        StoppedRpcEngine.class, RpcEngine.class);
+  }
 
 
   int datasize = 1024*100;
   int datasize = 1024*100;
   int numThreads = 50;
   int numThreads = 50;
-
-  public TestRPC(String name) { super(name); }
 	
 	
   public interface TestProtocol extends VersionedProtocol {
   public interface TestProtocol extends VersionedProtocol {
     public static final long versionID = 1L;
     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 {
   public void testConfRpc() throws Exception {
     Server server = RPC.getServer(TestProtocol.class,
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, 1, false, conf, null);
                                   new TestImpl(), ADDRESS, 0, 1, false, conf, null);
@@ -229,6 +311,7 @@ public class TestRPC extends TestCase {
     server.stop();    
     server.stop();    
   }
   }
 
 
+  @Test
   public void testSlowRpc() throws Exception {
   public void testSlowRpc() throws Exception {
     System.out.println("Testing Slow RPC");
     System.out.println("Testing Slow RPC");
     // create a server with two handlers
     // 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,
     Server server = RPC.getServer(TestProtocol.class,
                                   new TestImpl(), ADDRESS, 0, conf);
                                   new TestImpl(), ADDRESS, 0, conf);
     TestProtocol proxy = null;
     TestProtocol proxy = null;
@@ -384,6 +468,7 @@ public class TestRPC extends TestCase {
     }
     }
   }
   }
   
   
+  @Test
   public void testStandaloneClient() throws IOException {
   public void testStandaloneClient() throws IOException {
     try {
     try {
       TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
       TestProtocol proxy = RPC.waitForProxy(TestProtocol.class,
@@ -450,6 +535,7 @@ public class TestRPC extends TestCase {
     }
     }
   }
   }
   
   
+  @Test
   public void testAuthorization() throws Exception {
   public void testAuthorization() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
     conf.setBoolean(CommonConfigurationKeys.HADOOP_SECURITY_AUTHORIZATION,
@@ -481,20 +567,48 @@ public class TestRPC extends TestCase {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
     
     
     conf.setBoolean("ipc.client.ping", false);
     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);
     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
    * Test stopping a non-registered proxy
    * @throws Exception
    * @throws Exception
    */
    */
+  @Test
   public void testStopNonRegisteredProxy() throws Exception {
   public void testStopNonRegisteredProxy() throws Exception {
     RPC.stopProxy(mock(TestProtocol.class));
     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 {
   public void testErrorMsgForInsecureClient() throws Exception {
     final Server server = RPC.getServer(TestProtocol.class,
     final Server server = RPC.getServer(TestProtocol.class,
         new TestImpl(), ADDRESS, 0, 5, true, conf, null);
         new TestImpl(), ADDRESS, 0, 5, true, conf, null);
@@ -567,10 +681,10 @@ public class TestRPC extends TestCase {
     return count;
     return count;
   }
   }
 
 
-
   /**
   /**
    * Test that server.stop() properly stops all threads
    * Test that server.stop() properly stops all threads
    */
    */
+  @Test
   public void testStopsAllThreads() throws Exception {
   public void testStopsAllThreads() throws Exception {
     int threadsBefore = countThreads("Server$Listener$Reader");
     int threadsBefore = countThreads("Server$Listener$Reader");
     assertEquals("Expect no Reader threads running before test",
     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 {
   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-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
 Release 0.23.0 - Unreleased
 
 
   INCOMPATIBLE CHANGES
   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.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 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.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.UpgradeStatusReport;
 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.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
@@ -797,12 +797,20 @@ public class DFSClient implements java.io.Closeable {
    * 
    * 
    * @param src file name
    * @param src file name
    * @param buffersize buffer size
    * @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
    * @return an output stream for writing into the file
    * 
    * 
    * @see ClientProtocol#append(String, String) 
    * @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 {
       throws IOException {
     checkOpen();
     checkOpen();
     HdfsFileStatus stat = getFileInfo(src);
     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.List;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FSOutputSummer;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
@@ -98,7 +99,8 @@ import org.apache.hadoop.util.PureJavaCrc32;
  * datanode from the original pipeline. The DataStreamer now
  * datanode from the original pipeline. The DataStreamer now
  * starts sending packets from the dataQueue.
  * 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 final DFSClient dfsClient;
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
   private Socket s;
   private Socket s;
@@ -1537,7 +1539,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable {
    * write pipeline have failed. 
    * write pipeline have failed. 
    * @return the number of valid replicas of the current block
    * @return the number of valid replicas of the current block
    */
    */
-  public synchronized int getNumCurrentReplicas() throws IOException {
+  synchronized int getNumCurrentReplicas() throws IOException {
     dfsClient.checkOpen();
     dfsClient.checkOpen();
     isClosed();
     isClosed();
     if (streamer == null) {
     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
    * Returns the size of a file as it was when this stream was opened
    */
    */
-  public long getInitialLen() {
+  long getInitialLen() {
     return initialFileSize;
     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
   @Override
   public FSDataOutputStream append(Path f, int bufferSize,
   public FSDataOutputStream append(Path f, int bufferSize,
       Progressable progress) throws IOException {
       Progressable progress) throws IOException {
-
     statistics.incrementWriteOps(1);
     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
   @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.fs.FSDataOutputStream;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
 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.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@@ -158,10 +157,8 @@ public class DatanodeWebHdfsMethods {
       final Configuration conf = new Configuration(datanode.getConf());
       final Configuration conf = new Configuration(datanode.getConf());
       final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
       final InetSocketAddress nnRpcAddr = NameNode.getAddress(conf);
       final DFSClient dfsclient = new DFSClient(nnRpcAddr, 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 {
       try {
         IOUtils.copyBytes(in, out, bufferSize.getValue());
         IOUtils.copyBytes(in, out, bufferSize.getValue());
       } finally {
       } 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);
     final HttpURLConnection conn = httpConnect(op, fspath, parameters);
     validateResponse(op, conn);
     validateResponse(op, conn);
     try {
     try {
-      return jsonParse(conn.getInputStream());
+      return WebHdfsFileSystem.<T>jsonParse(conn.getInputStream());
     } finally {
     } finally {
       conn.disconnect();
       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
     org.apache.hadoop.yarn.api.records.* to be get/set only. Added javadocs to
     all public records. (acmurthy)
     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
   OPTIMIZATIONS
 
 
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
     MAPREDUCE-2026. Make JobTracker.getJobCounters() and
@@ -1234,83 +1246,92 @@ Release 0.23.0 - Unreleased
     MAPREDUCE-2897. Javadoc for ClientRMProtocol protocol and related records. 
     MAPREDUCE-2897. Javadoc for ClientRMProtocol protocol and related records. 
     (acmurthy)
     (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
 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-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
 Release 0.21.1 - Unreleased
 
 
   NEW FEATURES
   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) {
   public static void main(String[] args) {
     try {
     try {
       //Configuration.addDefaultResource("job.xml");
       //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.setClusterTimestamp(Long.valueOf(args[0]));
       applicationId.setId(Integer.valueOf(args[1]));
       applicationId.setId(Integer.valueOf(args[1]));
       int failCount = Integer.valueOf(args[2]);
       int failCount = Integer.valueOf(args[2]);
       MRAppMaster appMaster = new MRAppMaster(applicationId, failCount);
       MRAppMaster appMaster = new MRAppMaster(applicationId, failCount);
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(appMaster));
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
       conf.addResource(new Path(MRConstants.JOB_CONF_FILE));
       conf.addResource(new Path(MRConstants.JOB_CONF_FILE));
       conf.set(MRJobConfig.USER_NAME, 
       conf.set(MRJobConfig.USER_NAME, 
@@ -573,7 +575,7 @@ public class MRAppMaster extends CompositeService {
       appMaster.init(conf);
       appMaster.init(conf);
       appMaster.start();
       appMaster.start();
     } catch (Throwable t) {
     } catch (Throwable t) {
-      LOG.error("Caught throwable. Exiting:", t);
+      LOG.fatal("Error starting MRAppMaster", t);
       System.exit(1);
       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.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 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.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 
 
 
 
 /**
 /**
@@ -52,6 +54,16 @@ public interface Job {
   int getCompletedReduces();
   int getCompletedReduces();
   boolean isUber();
   boolean isUber();
   String getUserName();
   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[]
   TaskAttemptCompletionEvent[]
       getTaskAttemptCompletionEvents(int fromEventId, int maxEvents);
       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();
   float getProgress();
   TaskAttemptState getState();
   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();
   boolean isFinished();
 
 
-  /**If container Assigned then return container ID, otherwise null.
+  /**
+   * @return the container ID if a container is assigned, otherwise null.
    */
    */
   ContainerId getAssignedContainerID();
   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();
   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();
   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.
    * yet, returns 0.
    */
    */
   long getLaunchTime();
   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.
    *  yet, returns 0.
    */
    */
   long getFinishTime();
   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.
    * @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;
     return userName;
   }
   }
   
   
+  /*
+   * (non-Javadoc)
+   * @see org.apache.hadoop.mapreduce.v2.app.job.Job#getConfFile()
+   */
+  @Override
+  public Path getConfFile() {
+    return remoteJobConfFile;
+  }
+  
   @Override
   @Override
   public String getName() {
   public String getName() {
     return jobName;
     return jobName;
@@ -787,6 +796,15 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
   public int getTotalReduces() {
   public int getTotalReduces() {
     return reduceTasks.size();  //FIXME: why indirection? return numReduceTasks
     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 
   public static class InitTransition 
       implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
       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());
           + remoteJobConfPath.toUri().toASCIIString());
       // //////////// End of JobConf setup
       // //////////// End of JobConf setup
 
 
-      
       // Setup DistributedCache
       // Setup DistributedCache
-      setupDistributedCache(remoteFS, conf, localResources, environment);
+      MRApps.setupDistributedCache(conf, localResources, environment);
 
 
       // Set local-resources and environment
       // Set local-resources and environment
       container.setLocalResources(localResources);
       container.setLocalResources(localResources);
-      container.setEnv(environment);
+      container.setEnvironment(environment);
       
       
       // Setup up tokens
       // Setup up tokens
       Credentials taskCredentials = new Credentials();
       Credentials taskCredentials = new Credentials();
@@ -618,7 +617,7 @@ public abstract class TaskAttemptImpl implements
           ShuffleHandler.serializeServiceData(jobToken));
           ShuffleHandler.serializeServiceData(jobToken));
       container.setServiceData(serviceData);
       container.setServiceData(serviceData);
 
 
-      MRApps.addToClassPath(container.getEnv(), getInitialClasspath());
+      MRApps.addToClassPath(container.getEnvironment(), getInitialClasspath());
     } catch (IOException e) {
     } catch (IOException e) {
       throw new YarnException(e);
       throw new YarnException(e);
     }
     }
@@ -645,7 +644,7 @@ public abstract class TaskAttemptImpl implements
         taskAttemptListener.getAddress(), remoteTask, javaHome,
         taskAttemptListener.getAddress(), remoteTask, javaHome,
         workDir.toString(), containerLogDir, childTmpDir, jvmID));
         workDir.toString(), containerLogDir, childTmpDir, jvmID));
 
 
-    MapReduceChildJVM.setVMEnv(container.getEnv(), classPaths,
+    MapReduceChildJVM.setVMEnv(container.getEnvironment(), classPaths,
         workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask,
         workDir.toString(), containerLogDir, nmLdLibraryPath, remoteTask,
         localizedApplicationTokensFile);
         localizedApplicationTokensFile);
 
 
@@ -656,116 +655,6 @@ public abstract class TaskAttemptImpl implements
     return container;
     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
   @Override
   public ContainerId getAssignedContainerID() {
   public ContainerId getAssignedContainerID() {
     readLock.lock();
     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
   @Override
   public int getShufflePort() {
   public int getShufflePort() {
@@ -862,6 +770,7 @@ public abstract class TaskAttemptImpl implements
       result.setProgress(reportedStatus.progress);
       result.setProgress(reportedStatus.progress);
       result.setStartTime(launchTime);
       result.setStartTime(launchTime);
       result.setFinishTime(finishTime);
       result.setFinishTime(finishTime);
+      result.setShuffleFinishTime(this.reportedStatus.shuffleFinishTime);
       result.setDiagnosticInfo(reportedStatus.diagnosticInfo);
       result.setDiagnosticInfo(reportedStatus.diagnosticInfo);
       result.setPhase(reportedStatus.phase);
       result.setPhase(reportedStatus.phase);
       result.setStateString(reportedStatus.stateString);
       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, " ",
         setTitle(join(attemptState, " ",
             MRApps.taskType(taskType).toString(), " attempts in ", $(JOB_ID)));
             MRApps.taskType(taskType).toString(), " attempts in ", $(JOB_ID)));
+
+        render(attemptsPage());
       } catch (Exception e) {
       } catch (Exception e) {
         badRequest(e.getMessage());
         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.
    * Ensure that a JOB_ID was passed into the page.
    */
    */
-  void requireJob() {
+  public void requireJob() {
     try {
     try {
       if ($(JOB_ID).isEmpty()) {
       if ($(JOB_ID).isEmpty()) {
         throw new RuntimeException("missing job ID");
         throw new RuntimeException("missing job ID");
@@ -216,14 +217,15 @@ public class AppController extends Controller implements AMParams {
         notFound($(JOB_ID));
         notFound($(JOB_ID));
       }
       }
     } catch (Exception e) {
     } 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.
    * Ensure that a TASK_ID was passed into the page.
    */
    */
-  void requireTask() {
+  public void requireTask() {
     try {
     try {
       if ($(TASK_ID).isEmpty()) {
       if ($(TASK_ID).isEmpty()) {
         throw new RuntimeException("missing task ID");
         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.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
 import java.util.Collection;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapred.ShuffleHandler;
 import org.apache.hadoop.mapreduce.FileSystemCounter;
 import org.apache.hadoop.mapreduce.FileSystemCounter;
 import org.apache.hadoop.mapreduce.JobACL;
 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.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.MockApps;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -258,6 +261,16 @@ public class MockJobs extends MockApps {
       public String getAssignedContainerMgrAddress() {
       public String getAssignedContainerMgrAddress() {
         return "localhost:9998";
         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
       @Override
       public List<String> getDiagnostics() {
       public List<String> getDiagnostics() {
-        throw new UnsupportedOperationException("Not supported yet.");
+        return Collections.<String>emptyList();
       }
       }
 
 
       @Override
       @Override
@@ -463,8 +476,18 @@ public class MockJobs extends MockApps {
 
 
       @Override
       @Override
       public String getUserName() {
       public String getUserName() {
+        return "mock";
+      }
+
+      @Override
+      public Path getConfFile() {
         throw new UnsupportedOperationException("Not supported yet.");
         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.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 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.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
 import org.apache.hadoop.mapreduce.v2.app.speculate.TaskRuntimeEstimator;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -461,6 +463,16 @@ public class TestRuntimeEstimators {
     public String getUserName() {
     public String getUserName() {
       throw new UnsupportedOperationException("Not supported yet.");
       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.");
       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
     @Override
     public String getAssignedContainerMgrAddress() {
     public String getAssignedContainerMgrAddress() {
       throw new UnsupportedOperationException("Not supported yet.");
       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.apache.hadoop.mapreduce.v2.app.webapp.AMParams.APP_ID;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 
 
+import java.util.HashMap;
 import java.util.Map;
 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.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.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 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.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.Clock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -115,18 +120,42 @@ public class TestAMWebApp {
     WebAppTests.testPage(AppView.class, AppContext.class, new TestAppContext());
     WebAppTests.testPage(AppView.class, AppContext.class, new TestAppContext());
   }
   }
 
 
+
+  
   @Test public void testJobView() {
   @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() {
   @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() {
   @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) {
   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 float getProgress();
   public abstract long getStartTime();
   public abstract long getStartTime();
   public abstract long getFinishTime();
   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 Counters getCounters();
   public abstract String getDiagnosticInfo();
   public abstract String getDiagnosticInfo();
   public abstract String getStateString();
   public abstract String getStateString();
@@ -39,4 +43,14 @@ public interface TaskAttemptReport {
   public abstract void setStateString(String stateString);
   public abstract void setStateString(String stateString);
   public abstract void setPhase(Phase phase);
   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();
     maybeInitBuilder();
     builder.setFinishTime((finishTime));
     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
   @Override
   public TaskAttemptId getTaskAttemptId() {
   public TaskAttemptId getTaskAttemptId() {
     TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
     TaskAttemptReportProtoOrBuilder p = viaProto ? proto : builder;
@@ -262,7 +287,4 @@ public class TaskAttemptReportPBImpl extends ProtoBase<TaskAttemptReportProto> i
   private Phase convertFromProtoFormat(PhaseProto e) {
   private Phase convertFromProtoFormat(PhaseProto e) {
     return MRProtoUtils.convertFromProtoFormat(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();
     maybeInitBuilder();
     builder.setStartTime((startTime));
     builder.setStartTime((startTime));
   }
   }
+  
   @Override
   @Override
   public long getFinishTime() {
   public long getFinishTime() {
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;
@@ -143,6 +144,7 @@ public class TaskReportPBImpl extends ProtoBase<TaskReportProto> implements Task
     maybeInitBuilder();
     maybeInitBuilder();
     builder.setFinishTime((finishTime));
     builder.setFinishTime((finishTime));
   }
   }
+  
   @Override
   @Override
   public TaskId getTaskId() {
   public TaskId getTaskId() {
     TaskReportProtoOrBuilder p = viaProto ? proto : builder;
     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.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
+import java.net.URI;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 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.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 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.MRConstants;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 import org.apache.hadoop.mapreduce.v2.api.records.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.util.Shell.ShellCommandExecutor;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 
 
 /**
 /**
  * Helper class for MR applications
  * Helper class for MR applications
  */
  */
+@Private
+@Unstable
 public class MRApps extends Apps {
 public class MRApps extends Apps {
   public static final String JOB = "job";
   public static final String JOB = "job";
   public static final String TASK = "task";
   public static final String TASK = "task";
@@ -232,4 +244,121 @@ public class MRApps extends Apps {
         jobId.toString() + Path.SEPARATOR + MRConstants.JOB_CONF_FILE);
         jobId.toString() + Path.SEPARATOR + MRConstants.JOB_CONF_FILE);
     return jobFile.toString();
     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 diagnostic_info = 7;
   optional string state_string = 8;
   optional string state_string = 8;
   optional PhaseProto phase = 9;
   optional PhaseProto phase = 9;
+  optional int64 shuffle_finish_time = 10;
+  optional int64 sort_finish_time=11;
 }
 }
 
 
 enum JobStateProto {
 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; }
     public long getSubmitTime() { return submitTime; }
-    /** Get the job finish time */
+    /** @return the job finish time */
     public long getFinishTime() { return finishTime; }
     public long getFinishTime() { return finishTime; }
-    /** Get the job id */
+    /** @return the job id */
     public JobID getJobId() { return jobid; }
     public JobID getJobId() { return jobid; }
-    /** Get the user name */
+    /** @return the user name */
     public String getUsername() { return username; }
     public String getUsername() { return username; }
-    /** Get the job name */
+    /** @return the job name */
     public String getJobname() { return jobname; }
     public String getJobname() { return jobname; }
-    /** Get the job queue name */
+    /** @return the job queue name */
     public String getJobQueueName() { return jobQueueName; }
     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; }
     public String getJobConfPath() { return jobConfPath; }
-    /** Get the job launch time */
+    /** @return the job launch time */
     public long getLaunchTime() { return launchTime; }
     public long getLaunchTime() { return launchTime; }
-    /** Get the total number of maps */
+    /** @return the total number of maps */
     public long getTotalMaps() { return totalMaps; }
     public long getTotalMaps() { return totalMaps; }
-    /** Get the total number of reduces */
+    /** @return the total number of reduces */
     public long getTotalReduces() { return totalReduces; }
     public long getTotalReduces() { return totalReduces; }
-    /** Get the total number of failed maps */
+    /** @return the total number of failed maps */
     public long getFailedMaps() { return failedMaps; }
     public long getFailedMaps() { return failedMaps; }
-    /** Get the number of failed reduces */
+    /** @return the number of failed reduces */
     public long getFailedReduces() { return failedReduces; }
     public long getFailedReduces() { return failedReduces; }
-    /** Get the number of finished maps */
+    /** @return the number of finished maps */
     public long getFinishedMaps() { return finishedMaps; }
     public long getFinishedMaps() { return finishedMaps; }
-    /** Get the number of finished reduces */
+    /** @return the number of finished reduces */
     public long getFinishedReduces() { return finishedReduces; }
     public long getFinishedReduces() { return finishedReduces; }
-    /** Get the job status */
+    /** @return the job status */
     public String getJobStatus() { return jobStatus; }
     public String getJobStatus() { return jobStatus; }
     public String getErrorInfo() { return errorInfo; }
     public String getErrorInfo() { return errorInfo; }
-    /** Get the counters for the job */
+    /** @return the counters for the job */
     public Counters getTotalCounters() { return totalCounters; }
     public Counters getTotalCounters() { return totalCounters; }
-    /** Get the map counters for the job */
+    /** @return the map counters for the job */
     public Counters getMapCounters() { return mapCounters; }
     public Counters getMapCounters() { return mapCounters; }
-    /** Get the reduce counters for the job */
+    /** @return the reduce counters for the job */
     public Counters getReduceCounters() { return reduceCounters; }
     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; }
     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 String getPriority() { return priority.toString(); }
     public Map<JobACL, AccessControlList> getJobACLs() { return jobACLs; }
     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; }
     public TaskID getTaskId() { return taskId; }
-    /** Get the start time of this task */
+    /** @return the start time of this task */
     public long getStartTime() { return startTime; }
     public long getStartTime() { return startTime; }
-    /** Get the finish time of this task */
+    /** @return the finish time of this task */
     public long getFinishTime() { return finishTime; }
     public long getFinishTime() { return finishTime; }
-    /** Get the task type */
+    /** @return the task type */
     public TaskType getTaskType() { return taskType; }
     public TaskType getTaskType() { return taskType; }
-    /** Get the split locations */
+    /** @return the split locations */
     public String getSplitLocations() { return splitLocations; }
     public String getSplitLocations() { return splitLocations; }
-    /** Get the counters for this task */
+    /** @return the counters for this task */
     public Counters getCounters() { return counters; }
     public Counters getCounters() { return counters; }
-    /** Get the task status */
+    /** @return the task status */
     public String getTaskStatus() { return 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() {
     public TaskAttemptID getFailedDueToAttemptId() {
       return failedDueToAttemptId;
       return failedDueToAttemptId;
     }
     }
-    /** Get the error */
+    /** @return the error */
     public String getError() { return 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() {
     public Map<TaskAttemptID, TaskAttemptInfo> getAllTaskAttempts() {
       return attemptsMap;
       return attemptsMap;
     }
     }
@@ -530,33 +530,33 @@ public class JobHistoryParser {
       }
       }
     }
     }
 
 
-    /** Get the attempt Id */
+    /** @return the attempt Id */
     public TaskAttemptID getAttemptId() { return attemptId; }
     public TaskAttemptID getAttemptId() { return attemptId; }
-    /** Get the start time of the attempt */
+    /** @return the start time of the attempt */
     public long getStartTime() { return startTime; }
     public long getStartTime() { return startTime; }
-    /** Get the finish time of the attempt */
+    /** @return the finish time of the attempt */
     public long getFinishTime() { return finishTime; }
     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; }
     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; }
     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; }
     public long getMapFinishTime() { return mapFinishTime; }
-    /** Get the error string */
+    /** @return the error string */
     public String getError() { return error; }
     public String getError() { return error; }
-    /** Get the state */
+    /** @return the state */
     public String getState() { return state; }
     public String getState() { return state; }
-    /** Get the task status */
+    /** @return the task status */
     public String getTaskStatus() { return status; }
     public String getTaskStatus() { return status; }
-    /** Get the task type */
+    /** @return the task type */
     public TaskType getTaskType() { return taskType; }
     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; }
     public String getTrackerName() { return trackerName; }
-    /** Get the host name */
+    /** @return the host name */
     public String getHostname() { return hostname; }
     public String getHostname() { return hostname; }
-    /** Get the counters for the attempt */
+    /** @return the counters for the attempt */
     public Counters getCounters() { return counters; }
     public Counters getCounters() { return counters; }
-    /** Get the HTTP port for the tracker */
+    /** @return the HTTP port for the tracker */
     public int getHttpPort() { return httpPort; }
     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;
 package org.apache.hadoop.mapreduce.jobhistory;
 
 
-import java.io.IOException;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.mapred.TaskStatus;
 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.mapreduce.TaskType;
 
 
 import org.apache.hadoop.mapred.ProgressSplitsBlock;
 import org.apache.hadoop.mapred.ProgressSplitsBlock;
-import org.apache.hadoop.mapred.TaskStatus;
 
 
 import org.apache.avro.util.Utf8;
 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> mapTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
   private final Map<TaskId, Task> reduceTasks = new HashMap<TaskId, Task>();
   private final String user;
   private final String user;
+  private final Path confFile;
   
   
   private List<TaskAttemptCompletionEvent> completionEvents = null;
   private List<TaskAttemptCompletionEvent> completionEvents = null;
   private JobInfo jobInfo;
   private JobInfo jobInfo;
 
 
   public CompletedJob(Configuration conf, JobId jobId, Path historyFile, 
   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);
     LOG.info("Loading job: " + jobId + " from file: " + historyFile);
     this.conf = conf;
     this.conf = conf;
     this.jobId = jobId;
     this.jobId = jobId;
+    this.confFile = confFile;
     
     
     loadFullHistoryData(loadTasks, historyFile);
     loadFullHistoryData(loadTasks, historyFile);
 
 
     user = userName;
     user = userName;
     counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
     counters = TypeConverter.toYarn(jobInfo.getTotalCounters());
     diagnostics.add(jobInfo.getErrorInfo());
     diagnostics.add(jobInfo.getErrorInfo());
-    report = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class);
+    report =
+        RecordFactoryProvider.getRecordFactory(null).newRecordInstance(
+            JobReport.class);
     report.setJobId(jobId);
     report.setJobId(jobId);
     report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
     report.setJobState(JobState.valueOf(jobInfo.getJobStatus()));
     report.setStartTime(jobInfo.getLaunchTime());
     report.setStartTime(jobInfo.getLaunchTime());
@@ -192,11 +196,12 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
 
 
       int attemptRunTime = -1;
       int attemptRunTime = -1;
       if (taskAttempt.getLaunchTime() != 0 && taskAttempt.getFinishTime() != 0) {
       if (taskAttempt.getLaunchTime() != 0 && taskAttempt.getFinishTime() != 0) {
-        attemptRunTime = (int) (taskAttempt.getFinishTime() - taskAttempt
-            .getLaunchTime());
+        attemptRunTime =
+            (int) (taskAttempt.getFinishTime() - taskAttempt.getLaunchTime());
       }
       }
       // Default to KILLED
       // Default to KILLED
-      TaskAttemptCompletionEventStatus taceStatus = TaskAttemptCompletionEventStatus.KILLED;
+      TaskAttemptCompletionEventStatus taceStatus =
+          TaskAttemptCompletionEventStatus.KILLED;
       String taStateString = taskAttempt.getState().toString();
       String taStateString = taskAttempt.getState().toString();
       try {
       try {
         taceStatus = TaskAttemptCompletionEventStatus.valueOf(taStateString);
         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
   //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 + "]");
     LOG.info("Loading history file: [" + historyFileAbsolute + "]");
     if (jobInfo != null) {
     if (jobInfo != null) {
       return; //data already loaded
       return; //data already loaded
@@ -230,11 +236,13 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     
     
     if (historyFileAbsolute != null) {
     if (historyFileAbsolute != null) {
       try {
       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) {
       } catch (IOException e) {
-        throw new YarnException("Could not load history file " + historyFileAbsolute,
-            e);
+        throw new YarnException("Could not load history file "
+            + historyFileAbsolute, e);
       }
       }
     } else {
     } else {
       throw new IOException("History file not found");
       throw new IOException("History file not found");
@@ -293,7 +301,8 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
   }
   }
 
 
   @Override
   @Override
-  public boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
+  public
+      boolean checkAccess(UserGroupInformation callerUGI, JobACL jobOperation) {
     if (!UserGroupInformation.isSecurityEnabled()) {
     if (!UserGroupInformation.isSecurityEnabled()) {
       return true;
       return true;
     }
     }
@@ -304,8 +313,26 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
         jobInfo.getUsername(), jobACL);
         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
   @Override
   public String getUserName() {
   public String getUserName() {
     return user;
     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.setStartTime(attemptInfo.getStartTime());
     
     
     report.setFinishTime(attemptInfo.getFinishTime());
     report.setFinishTime(attemptInfo.getFinishTime());
+    report.setShuffleFinishTime(attemptInfo.getShuffleFinishTime());
+    report.setSortFinishTime(attemptInfo.getSortFinishTime());
     if (localDiagMessage != null) {
     if (localDiagMessage != null) {
       report.setDiagnosticInfo(attemptInfo.getError() + ", " + localDiagMessage);
       report.setDiagnosticInfo(attemptInfo.getError() + ", " + localDiagMessage);
     } else {
     } else {
@@ -158,10 +160,19 @@ public class CompletedTaskAttempt implements TaskAttempt {
   public long getFinishTime() {
   public long getFinishTime() {
     return report.getFinishTime();
     return report.getFinishTime();
   }
   }
+  
+  @Override
+  public long getShuffleFinishTime() {
+    return report.getShuffleFinishTime();
+  }
+
+  @Override
+  public long getSortFinishTime() {
+    return report.getSortFinishTime();
+  }
 
 
   @Override
   @Override
   public int getShufflePort() {
   public int getShufflePort() {
     throw new UnsupportedOperationException("Not supported yet.");
     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.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.Calendar;
 import java.util.Collections;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashMap;
@@ -36,8 +35,6 @@ import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 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.Log;
 import org.apache.commons.logging.LogFactory;
 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 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
    * 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
     @Override
     public int compare(JobId o1, JobId o2) {
     public int compare(JobId o1, JobId o2) {
-      if (o1.getAppId().getClusterTimestamp() > o2.getAppId().getClusterTimestamp()) {
+      if (o1.getAppId().getClusterTimestamp() > 
+          o2.getAppId().getClusterTimestamp()) {
         return 1;
         return 1;
-      } else if (o1.getAppId().getClusterTimestamp() < o2.getAppId().getClusterTimestamp()) {
+      } else if (o1.getAppId().getClusterTimestamp() < 
+          o2.getAppId().getClusterTimestamp()) {
         return -1;
         return -1;
       } else {
       } else {
         return o1.getId() - o2.getId();
         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
    * 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.
    * Facilitates jobId based searches.
    * If a jobId is not found in this list - it will not be found.
    * 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).
   //Maintains minimal details for recent jobs (parsed from history file name).
   //Sorted on Job Completion Time.
   //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)
   // Re-use exisiting MetaInfo objects if they exist for the specific JobId. (synchronization on MetaInfo)
   // Check for existance of the object when using iterators.
   // 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.
   //Maintains a list of known done subdirectories. Not currently used.
   private final Set<Path> existingDoneSubdirs = new HashSet<Path>();
   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.
   //The number of jobs to maintain in the job list cache.
   private int jobListCacheSize;
   private int jobListCacheSize;
@@ -187,7 +188,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
     debugMode = conf.getBoolean(JHAdminConfig.MR_HISTORY_DEBUG_MODE, false);
     serialNumberLowDigits = debugMode ? 1 : 3;
     serialNumberLowDigits = debugMode ? 1 : 3;
     serialNumberFormat = ("%0"
     serialNumberFormat = ("%0"
-        + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS + serialNumberLowDigits) + "d");
+        + (JobHistoryUtils.SERIAL_NUMBER_DIRECTORY_DIGITS 
+            + serialNumberLowDigits) + "d");
 
 
     String doneDirPrefix = null;
     String doneDirPrefix = null;
     doneDirPrefix = JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
     doneDirPrefix = JobHistoryUtils.getConfiguredHistoryServerDoneDirPrefix(conf);
@@ -195,9 +197,11 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
       doneDirPrefixPath = FileContext.getFileContext(conf).makeQualified(
           new Path(doneDirPrefix));
           new Path(doneDirPrefix));
       doneDirFc = FileContext.getFileContext(doneDirPrefixPath.toUri(), conf);
       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) {
     } catch (IOException e) {
-      throw new YarnException("Error creating done directory: [" + doneDirPrefixPath + "]", e);
+      throw new YarnException("Error creating done directory: [" +
+          doneDirPrefixPath + "]", e);
     }
     }
 
 
     String intermediateDoneDirPrefix = null;
     String intermediateDoneDirPrefix = null;
@@ -208,21 +212,27 @@ public class JobHistory extends AbstractService implements HistoryContext   {
           .makeQualified(new Path(intermediateDoneDirPrefix));
           .makeQualified(new Path(intermediateDoneDirPrefix));
       intermediateDoneDirFc = FileContext.getFileContext(
       intermediateDoneDirFc = FileContext.getFileContext(
           intermediateDoneDirPath.toUri(), conf);
           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) {
     } catch (IOException e) {
       LOG.info("error creating done directory on dfs " + 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 =
     moveThreadInterval =
         conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS,
         conf.getLong(JHAdminConfig.MR_HISTORY_MOVE_INTERVAL_MS,
             DEFAULT_MOVE_THREAD_INTERVAL);
             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 {
     try {
       initExisting();
       initExisting();
     } catch (IOException e) {
     } catch (IOException e) {
@@ -254,19 +264,21 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   @Override
   @Override
   public void start() {
   public void start() {
     //Start moveIntermediatToDoneThread
     //Start moveIntermediatToDoneThread
-    moveIntermediateToDoneRunnable = new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
+    moveIntermediateToDoneRunnable = 
+      new MoveIntermediateToDoneRunnable(moveThreadInterval, numMoveThreads);
     moveIntermediateToDoneThread = new Thread(moveIntermediateToDoneRunnable);
     moveIntermediateToDoneThread = new Thread(moveIntermediateToDoneRunnable);
     moveIntermediateToDoneThread.setName("MoveIntermediateToDoneScanner");
     moveIntermediateToDoneThread.setName("MoveIntermediateToDoneScanner");
     moveIntermediateToDoneThread.start();
     moveIntermediateToDoneThread.start();
     
     
     //Start historyCleaner
     //Start historyCleaner
-    boolean startCleanerService = conf.getBoolean(JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
+    boolean startCleanerService = conf.getBoolean(
+        JHAdminConfig.MR_HISTORY_CLEANER_ENABLE, true);
     if (startCleanerService) {
     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);
     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
       cleanerScheduledExecutor
           .scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles),
           .scheduleAtFixedRate(new HistoryCleaner(maxAgeOfHistoryFiles),
               30 * 1000l, runInterval, TimeUnit.MILLISECONDS);
               30 * 1000l, runInterval, TimeUnit.MILLISECONDS);
@@ -331,13 +343,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
   
   
   private void removeDirectoryFromSerialNumberIndex(Path serialDirPath) {
   private void removeDirectoryFromSerialNumberIndex(Path serialDirPath) {
     String serialPart = serialDirPath.getName();
     String serialPart = serialDirPath.getName();
-    String timeStampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
+    String timeStampPart = 
+      JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
     if (timeStampPart == null) {
     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;
       return;
     }
     }
     if (serialPart == null) {
     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;
       return;
     }
     }
     if (idToDateString.containsKey(serialPart)) {
     if (idToDateString.containsKey(serialPart)) {
@@ -355,13 +370,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       LOG.debug("Adding "+serialDirPath+" to serial index");
       LOG.debug("Adding "+serialDirPath+" to serial index");
     }
     }
     String serialPart = serialDirPath.getName();
     String serialPart = serialDirPath.getName();
-    String timestampPart = JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
+    String timestampPart = 
+      JobHistoryUtils.getTimestampPartFromPath(serialDirPath.toString());
     if (timestampPart == null) {
     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;
       return;
     }
     }
     if (serialPart == null) {
     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);
     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);
     path = fc.makeQualified(path);
     List<FileStatus> jhStatusList = new ArrayList<FileStatus>();
     List<FileStatus> jhStatusList = new ArrayList<FileStatus>();
       RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
       RemoteIterator<FileStatus> fileStatusIter = fc.listStatus(path);
@@ -414,7 +433,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     return jhStatusList;
     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());
     return scanDirectory(path, fc, JobHistoryUtils.getHistoryFileFilter());
   }
   }
   
   
@@ -425,7 +445,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * @return
    * @return
    */
    */
   private List<FileStatus> findTimestampedDirectories() throws IOException {
   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;
     return fsList;
   }
   }
     
     
@@ -434,7 +455,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    */
    */
   private void addToJobListCache(JobId jobId, MetaInfo metaInfo) {
   private void addToJobListCache(JobId jobId, MetaInfo metaInfo) {
     if(LOG.isDebugEnabled()) {
     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);
     jobListCache.put(jobId, metaInfo);
     if (jobListCache.size() > jobListCacheSize) {
     if (jobListCache.size() > jobListCacheSize) {
@@ -462,14 +484,16 @@ public class JobHistory extends AbstractService implements HistoryContext   {
    * @throws IOException
    * @throws IOException
    */
    */
   private void scanIntermediateDirectory() 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) {
     for (FileStatus userDir : userDirList) {
       String name = userDir.getPath().getName();
       String name = userDir.getPath().getName();
       long newModificationTime = userDir.getModificationTime();
       long newModificationTime = userDir.getModificationTime();
       boolean shouldScan = false;
       boolean shouldScan = false;
       synchronized (userDirModificationTimeMap) {
       synchronized (userDirModificationTimeMap) {
-        if (!userDirModificationTimeMap.containsKey(name) || newModificationTime > userDirModificationTimeMap.get(name)) {
+        if (!userDirModificationTimeMap.containsKey(name) || newModificationTime
+            > userDirModificationTimeMap.get(name)) {
             shouldScan = true;
             shouldScan = true;
             userDirModificationTimeMap.put(name, newModificationTime);
             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.
    * @return A MetaInfo object for the jobId, null if not found.
    * @throws IOException
    * @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) {
     for (FileStatus fs : fileStatusList) {
-      JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
+      JobIndexInfo jobIndexInfo = 
+        FileNameIndexUtils.getIndexInfo(fs.getPath().getName());
       if (jobIndexInfo.getJobId().equals(jobId)) {
       if (jobIndexInfo.getJobId().equals(jobId)) {
         String confFileName = JobHistoryUtils
         String confFileName = JobHistoryUtils
             .getIntermediateConfFileName(jobIndexInfo.getJobId());
             .getIntermediateConfFileName(jobIndexInfo.getJobId());
@@ -549,7 +575,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     }
     }
     for (String timestampPart : dateStringSet) {
     for (String timestampPart : dateStringSet) {
       Path logDir = canonicalHistoryLogPath(jobId, timestampPart);
       Path logDir = canonicalHistoryLogPath(jobId, timestampPart);
-      List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir, doneDirFc);
+      List<FileStatus> fileStatusList = scanDirectoryForHistoryFiles(logDir,
+          doneDirFc);
       MetaInfo metaInfo = getJobMetaInfo(fileStatusList, jobId);
       MetaInfo metaInfo = getJobMetaInfo(fileStatusList, jobId);
       if (metaInfo != null) {
       if (metaInfo != null) {
         return metaInfo;
         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
    * @param jobId
    * @return
    * @return
    * @throws IOException
    * @throws IOException
@@ -586,7 +614,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     
     
     MoveIntermediateToDoneRunnable(long sleepTime, int numMoveThreads) {
     MoveIntermediateToDoneRunnable(long sleepTime, int numMoveThreads) {
       this.sleepTime = sleepTime;
       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;
       running = true;
     }
     }
   
   
@@ -604,7 +633,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
                 try {
                 try {
                 moveToDone(metaInfo);
                 moveToDone(metaInfo);
                 } catch (IOException e) {
                 } 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) {
     synchronized(metaInfo) {
       try {
       try {
         Job job = new CompletedJob(conf, metaInfo.getJobIndexInfo().getJobId(), 
         Job job = new CompletedJob(conf, metaInfo.getJobIndexInfo().getJobId(), 
-            metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser());
+            metaInfo.getHistoryFile(), true, metaInfo.getJobIndexInfo().getUser(),
+            metaInfo.getConfFile());
         addToLoadedJobCache(job);
         addToLoadedJobCache(job);
         return job;
         return job;
       } catch (IOException e) {
       } 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() {
   private Map<JobId, Job> getAllJobsInternal() {
     //TODO This should ideally be using getAllJobsMetaInfo
     //TODO This should ideally be using getAllJobsMetaInfo
     // or get rid of that method once Job has APIs for user, finishTime etc.
     // 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;
     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 {
   private void moveToDone(MetaInfo metaInfo) throws IOException {
     long completeTime = metaInfo.getJobIndexInfo().getFinishTime();
     long completeTime = metaInfo.getJobIndexInfo().getFinishTime();
     if (completeTime == 0) completeTime = System.currentTimeMillis();
     if (completeTime == 0) completeTime = System.currentTimeMillis();
@@ -890,26 +797,31 @@ public class JobHistory extends AbstractService implements HistoryContext   {
     try {
     try {
       maybeMakeSubdirectory(targetDir);
       maybeMakeSubdirectory(targetDir);
     } catch (IOException e) {
     } 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;
       throw e;
     }
     }
     synchronized (metaInfo) {
     synchronized (metaInfo) {
       if (historyFile != null) {
       if (historyFile != null) {
-        Path toPath = doneDirFc.makeQualified(new Path(targetDir, historyFile.getName()));
+        Path toPath = doneDirFc.makeQualified(new Path(targetDir, 
+            historyFile.getName()));
         try {
         try {
           moveToDoneNow(historyFile, toPath);
           moveToDoneNow(historyFile, toPath);
         } catch (IOException e) {
         } 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;
           throw e;
         }
         }
         metaInfo.setHistoryFile(toPath);
         metaInfo.setHistoryFile(toPath);
       }
       }
       if (confFile != null) {
       if (confFile != null) {
-        Path toPath = doneDirFc.makeQualified(new Path(targetDir, confFile.getName()));
+        Path toPath = doneDirFc.makeQualified(new Path(targetDir, 
+            confFile.getName()));
         try {
         try {
           moveToDoneNow(confFile, toPath);
           moveToDoneNow(confFile, toPath);
         } catch (IOException e) {
         } 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;
           throw e;
         }
         }
         metaInfo.setConfFile(toPath);
         metaInfo.setConfFile(toPath);
@@ -953,7 +865,8 @@ public class JobHistory extends AbstractService implements HistoryContext   {
       }
       }
     } catch (FileNotFoundException fnfE) {
     } catch (FileNotFoundException fnfE) {
       try {
       try {
-        FsPermission fsp = new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
+        FsPermission fsp = 
+          new FsPermission(JobHistoryUtils.HISTORY_DONE_DIR_PERMISSION);
         doneDirFc.mkdir(path, fsp, true);
         doneDirFc.mkdir(path, fsp, true);
         FileStatus fsStatus = doneDirFc.getFileStatus(path);
         FileStatus fsStatus = doneDirFc.getFileStatus(path);
         LOG.info("Perms after creating " + fsStatus.getPermission().toShort()
         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) {
   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) {
   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;
     private Path summaryFile;
     JobIndexInfo jobIndexInfo;
     JobIndexInfo jobIndexInfo;
 
 
-    MetaInfo(Path historyFile, Path confFile, Path summaryFile, JobIndexInfo jobIndexInfo) {
+    MetaInfo(Path historyFile, Path confFile, Path summaryFile, 
+        JobIndexInfo jobIndexInfo) {
       this.historyFile = historyFile;
       this.historyFile = historyFile;
       this.confFile = confFile;
       this.confFile = confFile;
       this.summaryFile = summaryFile;
       this.summaryFile = summaryFile;
       this.jobIndexInfo = jobIndexInfo;
       this.jobIndexInfo = jobIndexInfo;
-      }
+    }
 
 
     Path getHistoryFile() { return historyFile; }
     Path getHistoryFile() { return historyFile; }
     Path getConfFile() { return confFile; }
     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
         //Sort in ascending order. Relies on YYYY/MM/DD/Serial
         Collections.sort(serialDirList);
         Collections.sort(serialDirList);
         for (FileStatus serialDir : serialDirList) {
         for (FileStatus serialDir : serialDirList) {
-          List<FileStatus> historyFileList = scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
+          List<FileStatus> historyFileList = 
+            scanDirectoryForHistoryFiles(serialDir.getPath(), doneDirFc);
           for (FileStatus historyFile : historyFileList) {
           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)) {
             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);
               delete(metaInfo);
             } else {
             } else {
               halted = true;
               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) {
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
     StringUtils.startupShutdownMessage(JobHistoryServer.class, args, LOG);
-    JobHistoryServer server = null;
     try {
     try {
-      server = new JobHistoryServer();
+      JobHistoryServer jobHistoryServer = new JobHistoryServer();
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(jobHistoryServer));
       YarnConfiguration conf = new YarnConfiguration(new JobConf());
       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);
       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.List;
 import java.util.Map;
 import java.util.Map;
 
 
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 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.app.job.Task;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobIndexInfo;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 
 
 import clover.org.apache.log4j.Logger;
 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();
     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() {
   protected Class<? extends View> taskPage() {
     return HsTaskPage.class;
     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
   // Need all of these methods here also as Guice doesn't look into parent
   // classes.
   // classes.
   
   
@@ -127,6 +136,21 @@ public class HsController extends AppController {
     super.attempts();
     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.
    * @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 com.google.inject.Inject;
 import java.util.Date;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 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.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 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.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.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.job.Job;
 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.app.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 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.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
 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.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
 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.*;
 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 {
 public class HsJobBlock extends HtmlBlock {
   final AppContext appContext;
   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 killedMapAttempts = 0;
   int failedMapAttempts = 0;
   int failedMapAttempts = 0;
   int successfulMapAttempts = 0;
   int successfulMapAttempts = 0;
-  int newReduceAttempts = 0;
-  int runningReduceAttempts = 0;
   int killedReduceAttempts = 0;
   int killedReduceAttempts = 0;
   int failedReduceAttempts = 0;
   int failedReduceAttempts = 0;
   int successfulReduceAttempts = 0;
   int successfulReduceAttempts = 0;
+  long avgMapTime = 0;
+  long avgReduceTime = 0;
+  long avgShuffleTime = 0;
+  long avgSortTime = 0;
+  int numMaps;
+  int numReduces;
 
 
   @Inject HsJobBlock(AppContext appctx) {
   @Inject HsJobBlock(AppContext appctx) {
     appContext = appctx;
     appContext = appctx;
@@ -84,9 +85,9 @@ public class HsJobBlock extends HtmlBlock {
         p()._("Sorry, ", jid, " not found.")._();
         p()._("Sorry, ", jid, " not found.")._();
       return;
       return;
     }
     }
+    Map<JobACL, AccessControlList> acls = job.getJobACLs();
+    
     JobReport jobReport = job.getReport();
     JobReport jobReport = job.getReport();
-    String mapPct = percent(jobReport.getMapProgress());
-    String reducePct = percent(jobReport.getReduceProgress());
     int mapTasks = job.getTotalMaps();
     int mapTasks = job.getTotalMaps();
     int mapTasksComplete = job.getCompletedMaps();
     int mapTasksComplete = job.getCompletedMaps();
     int reduceTasks = job.getTotalReduces();
     int reduceTasks = job.getTotalReduces();
@@ -94,13 +95,38 @@ public class HsJobBlock extends HtmlBlock {
     long startTime = jobReport.getStartTime();
     long startTime = jobReport.getStartTime();
     long finishTime = jobReport.getFinishTime();
     long finishTime = jobReport.getFinishTime();
     countTasksAndAttempts(job);
     countTasksAndAttempts(job);
-    info("Job Overview").
+    ResponseInfo infoBlock = info("Job Overview").
         _("Job Name:", job.getName()).
         _("Job Name:", job.getName()).
+        _("User Name:", job.getUserName()).
         _("State:", job.getState()).
         _("State:", job.getState()).
         _("Uberized:", job.isUber()).
         _("Uberized:", job.isUber()).
         _("Started:", new Date(startTime)).
         _("Started:", new Date(startTime)).
+        _("Finished:", new Date(finishTime)).
         _("Elapsed:", StringUtils.formatTime(
         _("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.
     html.
       _(InfoBlock.class).
       _(InfoBlock.class).
       div(_INFO_WRAP).
       div(_INFO_WRAP).
@@ -109,34 +135,17 @@ public class HsJobBlock extends HtmlBlock {
         table("#job").
         table("#job").
           tr().
           tr().
             th(_TH, "Task Type").
             th(_TH, "Task Type").
-            th(_TH, "Progress").
             th(_TH, "Total").
             th(_TH, "Total").
-            th(_TH, "Pending").
-            th(_TH, "Running").
             th(_TH, "Complete")._().
             th(_TH, "Complete")._().
           tr(_ODD).
           tr(_ODD).
             th().
             th().
               a(url("tasks", jid, "m"), "Map")._().
               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(mapTasks)).
-            td(String.valueOf(pendingMapTasks)).
-            td(String.valueOf(runningMapTasks)).
             td(String.valueOf(mapTasksComplete))._().
             td(String.valueOf(mapTasksComplete))._().
           tr(_EVEN).
           tr(_EVEN).
             th().
             th().
               a(url("tasks", jid, "r"), "Reduce")._().
               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(reduceTasks)).
-            td(String.valueOf(pendingReduceTasks)).
-            td(String.valueOf(runningReduceTasks)).
             td(String.valueOf(reducesTasksComplete))._()
             td(String.valueOf(reducesTasksComplete))._()
           ._().
           ._().
 
 
@@ -144,19 +153,11 @@ public class HsJobBlock extends HtmlBlock {
         table("#job").
         table("#job").
         tr().
         tr().
           th(_TH, "Attempt Type").
           th(_TH, "Attempt Type").
-          th(_TH, "New").
-          th(_TH, "Running").
           th(_TH, "Failed").
           th(_TH, "Failed").
           th(_TH, "Killed").
           th(_TH, "Killed").
           th(_TH, "Successful")._().
           th(_TH, "Successful")._().
         tr(_ODD).
         tr(_ODD).
           th("Maps").
           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",
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.FAILED.toString()), 
               TaskAttemptStateUI.FAILED.toString()), 
               String.valueOf(failedMapAttempts))._().
               String.valueOf(failedMapAttempts))._().
@@ -169,12 +170,6 @@ public class HsJobBlock extends HtmlBlock {
         _().
         _().
         tr(_EVEN).
         tr(_EVEN).
           th("Reduces").
           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",
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.FAILED.toString()), 
               TaskAttemptStateUI.FAILED.toString()), 
               String.valueOf(failedReduceAttempts))._().
               String.valueOf(failedReduceAttempts))._().
@@ -195,44 +190,21 @@ public class HsJobBlock extends HtmlBlock {
    * @param job the job to get counts for.
    * @param job the job to get counts for.
    */
    */
   private void countTasksAndAttempts(Job job) {
   private void countTasksAndAttempts(Job job) {
+    numReduces = 0;
+    numMaps = 0;
     Map<TaskId, Task> tasks = job.getTasks();
     Map<TaskId, Task> tasks = job.getTasks();
     for (Task task : tasks.values()) {
     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
       // Attempts counts
       Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
       Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
       for (TaskAttempt attempt : attempts.values()) {
       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())) {
         if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
-          ++newAttempts;
+          //Do Nothing
         } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
         } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
             .getState())) {
             .getState())) {
-          ++running;
+          //Do Nothing
         } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
         } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
             .getState())) {
             .getState())) {
           ++successful;
           ++successful;
@@ -246,21 +218,41 @@ public class HsJobBlock extends HtmlBlock {
 
 
         switch (task.getType()) {
         switch (task.getType()) {
         case MAP:
         case MAP:
-          newMapAttempts += newAttempts;
-          runningMapAttempts += running;
           successfulMapAttempts += successful;
           successfulMapAttempts += successful;
           failedMapAttempts += failed;
           failedMapAttempts += failed;
           killedMapAttempts += killed;
           killedMapAttempts += killed;
+          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numMaps++;
+            avgMapTime += (attempt.getFinishTime() -
+                attempt.getLaunchTime());
+          }
           break;
           break;
         case REDUCE:
         case REDUCE:
-          newReduceAttempts += newAttempts;
-          runningReduceAttempts += running;
           successfulReduceAttempts += successful;
           successfulReduceAttempts += successful;
           failedReduceAttempts += failed;
           failedReduceAttempts += failed;
           killedReduceAttempts += killed;
           killedReduceAttempts += killed;
+          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numReduces++;
+            avgShuffleTime += (attempt.getShuffleFinishTime() - 
+                attempt.getLaunchTime());
+            avgSortTime += attempt.getSortFinishTime() - 
+                attempt.getLaunchTime();
+            avgReduceTime += (attempt.getFinishTime() -
+                attempt.getShuffleFinishTime());
+          }
           break;
           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().
         ul().
           li().a(url("job", jobid), "Overview")._().
           li().a(url("job", jobid), "Overview")._().
           li().a(url("jobcounters", jobid), "Counters")._().
           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, "m"), "Map tasks")._().
           li().a(url("tasks", jobid, "r"), "Reduce 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;
 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.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 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.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 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 static org.apache.hadoop.yarn.webapp.view.JQueryUI.tableInit;
 
 
 import java.util.Collection;
 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.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app.webapp.App;
 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;
 import org.apache.hadoop.util.StringUtils;
 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.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 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.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
 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.Hamlet.TR;
+import org.apache.hadoop.yarn.webapp.hamlet.HamletSpec.InputType;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
@@ -68,50 +72,162 @@ public class HsTaskPage extends HsView {
           h2($(TITLE));
           h2($(TITLE));
         return;
         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").
       table("#attempts").
         thead().
         thead().
-          tr().
+          tr();
+      
+      headRow.
             th(".id", "Attempt").
             th(".id", "Attempt").
-            th(".progress", "Progress").
             th(".state", "State").
             th(".state", "State").
             th(".node", "node").
             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()) {
       for (TaskAttempt ta : getTaskAttempts()) {
         String taid = MRApps.toString(ta.getID());
         String taid = MRApps.toString(ta.getID());
-        String progress = percent(ta.getProgress());
-        ContainerId containerId = ta.getAssignedContainerID();
 
 
         String nodeHttpAddr = ta.getNodeHttpAddress();
         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().
             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.
      * @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 up the java script and CSS for the attempts table
     set(DATATABLES_ID, "attempts");
     set(DATATABLES_ID, "attempts");
     set(initID(DATATABLES, "attempts"), attemptsTableInit());
     set(initID(DATATABLES, "attempts"), attemptsTableInit());
+    set(postInitID(DATATABLES, "attempts"), attemptsPostTableInit());
     setTableStyles(html, "attempts");
     setTableStyles(html, "attempts");
   }
   }
 
 
@@ -154,6 +271,49 @@ public class HsTaskPage extends HsView {
    * attempts table. 
    * attempts table. 
    */
    */
   private String attemptsTableInit() {
   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;
 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.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES;
 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.DATATABLES_ID;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 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 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;
 import org.apache.hadoop.yarn.webapp.SubView;
 
 
 /**
 /**
@@ -41,15 +44,16 @@ public class HsTasksPage extends HsView {
     set(DATATABLES_ID, "tasks");
     set(DATATABLES_ID, "tasks");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
     set(initID(ACCORDION, "nav"), "{autoHeight:false, active:1}");
     set(initID(DATATABLES, "tasks"), tasksTableInit());
     set(initID(DATATABLES, "tasks"), tasksTableInit());
+    set(postInitID(DATATABLES, "tasks"), jobsPostTableInit());
     setTableStyles(html, "tasks");
     setTableStyles(html, "tasks");
   }
   }
-
+  
   /**
   /**
    * The content of this page is the TasksBlock
    * The content of this page is the TasksBlock
-   * @return TasksBlock.class
+   * @return HsTasksBlock.class
    */
    */
   @Override protected Class<? extends SubView> content() {
   @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.
    * for the tasks table.
    */
    */
   private String tasksTableInit() {
   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("/", HsController.class);
     route("/app", HsController.class);
     route("/app", HsController.class);
     route(pajoin("/job", JOB_ID), HsController.class, "job");
     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("/jobcounters", JOB_ID), HsController.class, "jobCounters");
     route(pajoin("/tasks", JOB_ID, TASK_TYPE), HsController.class, "tasks");
     route(pajoin("/tasks", JOB_ID, TASK_TYPE), HsController.class, "tasks");
     route(pajoin("/attempts", JOB_ID, TASK_TYPE, ATTEMPT_STATE),
     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;
 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.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 static org.junit.Assert.assertEquals;
 
 
+import java.util.HashMap;
 import java.util.Map;
 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.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.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.MockJobs;
 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.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.Clock;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.inject.Injector;
 import com.google.inject.Injector;
 
 
 public class TestHSWebApp {
 public class TestHSWebApp {
+  private static final Log LOG = LogFactory.getLog(TestHSWebApp.class);
 
 
   static class TestAppContext implements AppContext {
   static class TestAppContext implements AppContext {
     final ApplicationAttemptId appAttemptID;
     final ApplicationAttemptId appAttemptID;
@@ -111,16 +122,53 @@ public class TestHSWebApp {
   }
   }
 
 
   @Test public void testJobView() {
   @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());
                          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;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
+import java.security.PrivilegedAction;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.Map;
 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.JobID;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.jobhistory.JHAdminConfig;
 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.net.NetUtils;
 import org.apache.hadoop.security.SecurityInfo;
 import org.apache.hadoop.security.SecurityInfo;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo;
 
 
 public class ClientCache {
 public class ClientCache {
 
 
@@ -72,16 +74,21 @@ public class ClientCache {
 
 
   private MRClientProtocol instantiateHistoryProxy()
   private MRClientProtocol instantiateHistoryProxy()
   throws IOException {
   throws IOException {
-	String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
+	final String serviceAddr = conf.get(JHAdminConfig.MR_HISTORY_ADDRESS,
 	          JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
 	          JHAdminConfig.DEFAULT_MR_HISTORY_ADDRESS);
     LOG.info("Connecting to HistoryServer at: " + serviceAddr);
     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,
     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);
     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;
 package org.apache.hadoop.mapred;
 
 
 import java.io.IOException;
 import java.io.IOException;
-import java.net.URI;
 import java.nio.ByteBuffer;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.HashMap;
 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.FSDataInputStream;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.io.DataOutputBuffer;
 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.TaskTrackerInfo;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.mapreduce.TypeConverter;
 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.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.mapreduce.v2.MRConstants;
 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.ApplicationReport;
 import org.apache.hadoop.yarn.api.records.ApplicationState;
 import org.apache.hadoop.yarn.api.records.ApplicationState;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 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.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 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
     // Construct necessary information to start the MR AM
     ApplicationSubmissionContext appContext = 
     ApplicationSubmissionContext appContext = 
       createApplicationSubmissionContext(conf, jobSubmitDir, ts);
       createApplicationSubmissionContext(conf, jobSubmitDir, ts);
-    setupDistributedCache(conf, appContext);
     
     
     // XXX Remove
     // XXX Remove
     in.close();
     in.close();
@@ -273,16 +270,18 @@ public class YARNRunner implements ClientProtocol {
   public ApplicationSubmissionContext createApplicationSubmissionContext(
   public ApplicationSubmissionContext createApplicationSubmissionContext(
       Configuration jobConf,
       Configuration jobConf,
       String jobSubmitDir, Credentials ts) throws IOException {
       String jobSubmitDir, Credentials ts) throws IOException {
-    ApplicationSubmissionContext appContext =
-        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     ApplicationId applicationId = resMgrDelegate.getApplicationId();
     ApplicationId applicationId = resMgrDelegate.getApplicationId();
-    appContext.setApplicationId(applicationId);
+    
+    // Setup resource requirements
     Resource capability = recordFactory.newRecordInstance(Resource.class);
     Resource capability = recordFactory.newRecordInstance(Resource.class);
     capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
     capability.setMemory(conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
         MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
         MRJobConfig.DEFAULT_MR_AM_VMEM_MB));
     LOG.info("AppMaster capability = " + capability);
     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);
     Path jobConfPath = new Path(jobSubmitDir, MRConstants.JOB_CONF_FILE);
     
     
     URL yarnUrlForJobSubmitDir = ConverterUtils
     URL yarnUrlForJobSubmitDir = ConverterUtils
@@ -292,14 +291,11 @@ public class YARNRunner implements ClientProtocol {
     LOG.debug("Creating setup context, jobSubmitDir url is "
     LOG.debug("Creating setup context, jobSubmitDir url is "
         + yarnUrlForJobSubmitDir);
         + yarnUrlForJobSubmitDir);
 
 
-    appContext.setResource(MRConstants.JOB_SUBMIT_DIR,
-        yarnUrlForJobSubmitDir);
-
-    appContext.setResourceTodo(MRConstants.JOB_CONF_FILE,
+    localResources.put(MRConstants.JOB_CONF_FILE,
         createApplicationResource(defaultFileContext,
         createApplicationResource(defaultFileContext,
             jobConfPath));
             jobConfPath));
     if (jobConf.get(MRJobConfig.JAR) != null) {
     if (jobConf.get(MRJobConfig.JAR) != null) {
-      appContext.setResourceTodo(MRConstants.JOB_JAR,
+      localResources.put(MRConstants.JOB_JAR,
           createApplicationResource(defaultFileContext,
           createApplicationResource(defaultFileContext,
               new Path(jobSubmitDir, MRConstants.JOB_JAR)));
               new Path(jobSubmitDir, MRConstants.JOB_JAR)));
     } else {
     } else {
@@ -312,30 +308,21 @@ public class YARNRunner implements ClientProtocol {
     // TODO gross hack
     // TODO gross hack
     for (String s : new String[] { "job.split", "job.splitmetainfo",
     for (String s : new String[] { "job.split", "job.splitmetainfo",
         MRConstants.APPLICATION_TOKENS_FILE }) {
         MRConstants.APPLICATION_TOKENS_FILE }) {
-      appContext.setResourceTodo(
+      localResources.put(
           MRConstants.JOB_SUBMIT_DIR + "/" + s,
           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";
     String javaHome = "$JAVA_HOME";
     Vector<CharSequence> vargs = new Vector<CharSequence>(8);
     Vector<CharSequence> vargs = new Vector<CharSequence>(8);
     vargs.add(javaHome + "/bin/java");
     vargs.add(javaHome + "/bin/java");
@@ -346,13 +333,6 @@ public class YARNRunner implements ClientProtocol {
     vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
     vargs.add(conf.get(MRJobConfig.MR_AM_COMMAND_OPTS,
         MRJobConfig.DEFAULT_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("org.apache.hadoop.mapreduce.v2.app.MRAppMaster");
     vargs.add(String.valueOf(applicationId.getClusterTimestamp()));
     vargs.add(String.valueOf(applicationId.getClusterTimestamp()));
     vargs.add(String.valueOf(applicationId.getId()));
     vargs.add(String.valueOf(applicationId.getId()));
@@ -370,140 +350,43 @@ public class YARNRunner implements ClientProtocol {
 
 
     LOG.info("Command to launch container for ApplicationMaster is : "
     LOG.info("Command to launch container for ApplicationMaster is : "
         + mergedCommand);
         + 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
   @Override
   public void setJobPriority(JobID arg0, String arg1) throws IOException,
   public void setJobPriority(JobID arg0, String arg1) throws IOException,
       InterruptedException {
       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;
 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.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 import org.apache.hadoop.yarn.api.ClientRMProtocol;
 
 
 /**
 /**
@@ -36,26 +30,17 @@ import org.apache.hadoop.yarn.api.ClientRMProtocol;
  * <p>It includes details such as:
  * <p>It includes details such as:
  *   <ul>
  *   <ul>
  *     <li>{@link ApplicationId} of the application.</li>
  *     <li>{@link ApplicationId} of the application.</li>
- *     <li>
- *       {@link Resource} necessary to run the <code>ApplicationMaster</code>.
- *     </li>
  *     <li>Application user.</li>
  *     <li>Application user.</li>
  *     <li>Application name.</li>
  *     <li>Application name.</li>
  *     <li>{@link Priority} of the application.</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>
  *     <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>
- *     <li>Command to launch the <code>ApplicationMaster</code>.</li>
  *   </ul>
  *   </ul>
  * </p>
  * </p>
  * 
  * 
+ * @see ContainerLaunchContext
  * @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)
  * @see ClientRMProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)
  */
  */
 @Public
 @Public
@@ -143,198 +128,25 @@ public interface ApplicationSubmissionContext {
   public void setUser(String user);
   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
   @Public
   @Stable
   @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
   @Public
   @Stable
   @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
   @Public
   @Stable
   @Stable
-  Map<String, String> getEnv();
+  Map<String, String> getEnvironment();
     
     
   /**
   /**
    * Add <em>environment variables</em> for the container.
    * Add <em>environment variables</em> for the container.
@@ -164,7 +164,7 @@ public interface ContainerLaunchContext {
    */
    */
   @Public
   @Public
   @Stable
   @Stable
-  void setEnv(Map<String, String> environment);
+  void setEnvironment(Map<String, String> environment);
 
 
   /**
   /**
    * Get the list of <em>commands</em> for launching the container.
    * 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;
 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.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 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.Priority;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
 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.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
 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.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;
   ApplicationSubmissionContextProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
   
   
   private ApplicationId applicationId = null;
   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 Priority priority = null;
-  
-  
+  private ContainerLaunchContext amContainer = null;
   
   
   public ApplicationSubmissionContextPBImpl() {
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
     builder = ApplicationSubmissionContextProto.newBuilder();
   }
   }
 
 
-  public ApplicationSubmissionContextPBImpl(ApplicationSubmissionContextProto proto) {
+  public ApplicationSubmissionContextPBImpl(
+      ApplicationSubmissionContextProto proto) {
     this.proto = proto;
     this.proto = proto;
     viaProto = true;
     viaProto = true;
   }
   }
@@ -83,30 +62,12 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     if (this.applicationId != null) {
     if (this.applicationId != null) {
       builder.setApplicationId(convertToProtoFormat(this.applicationId));
       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) {
     if (this.priority != null) {
       builder.setPriority(convertToProtoFormat(this.priority));
       builder.setPriority(convertToProtoFormat(this.priority));
     }
     }
+    if (this.amContainer != null) {
+      builder.setAmContainerSpec(convertToProtoFormat(this.amContainer));
+    }
   }
   }
 
 
   private void mergeLocalToProto() {
   private void mergeLocalToProto() {
@@ -145,6 +106,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
       builder.clearPriority();
       builder.clearPriority();
     this.priority = priority;
     this.priority = priority;
   }
   }
+  
   @Override
   @Override
   public ApplicationId getApplicationId() {
   public ApplicationId getApplicationId() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -165,6 +127,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
       builder.clearApplicationId();
       builder.clearApplicationId();
     this.applicationId = applicationId;
     this.applicationId = applicationId;
   }
   }
+  
   @Override
   @Override
   public String getApplicationName() {
   public String getApplicationName() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -183,403 +146,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     }
     }
     builder.setApplicationName((applicationName));
     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
   @Override
   public String getQueue() {
   public String getQueue() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -598,6 +165,7 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     }
     }
     builder.setQueue((queue));
     builder.setQueue((queue));
   }
   }
+  
   @Override
   @Override
   public String getUser() {
   public String getUser() {
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
     ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
@@ -617,6 +185,28 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     builder.setUser((user));
     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) {
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
     return new PriorityPBImpl(p);
   }
   }
@@ -633,28 +223,12 @@ public class ApplicationSubmissionContextPBImpl extends ProtoBase<ApplicationSub
     return ((ApplicationIdPBImpl)t).getProto();
     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.StringLocalResourceMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
 
 
-
-    
 public class ContainerLaunchContextPBImpl 
 public class ContainerLaunchContextPBImpl 
 extends ProtoBase<ContainerLaunchContextProto> 
 extends ProtoBase<ContainerLaunchContextProto> 
 implements ContainerLaunchContext {
 implements ContainerLaunchContext {
@@ -54,10 +52,9 @@ implements ContainerLaunchContext {
   private Map<String, LocalResource> localResources = null;
   private Map<String, LocalResource> localResources = null;
   private ByteBuffer containerTokens = null;
   private ByteBuffer containerTokens = null;
   private Map<String, ByteBuffer> serviceData = null;
   private Map<String, ByteBuffer> serviceData = null;
-  private Map<String, String> env = null;
+  private Map<String, String> environment = null;
   private List<String> commands = null;
   private List<String> commands = null;
   
   
-  
   public ContainerLaunchContextPBImpl() {
   public ContainerLaunchContextPBImpl() {
     builder = ContainerLaunchContextProto.newBuilder();
     builder = ContainerLaunchContextProto.newBuilder();
   }
   }
@@ -94,7 +91,7 @@ implements ContainerLaunchContext {
     if (this.serviceData != null) {
     if (this.serviceData != null) {
       addServiceDataToProto();
       addServiceDataToProto();
     }
     }
-    if (this.env != null) {
+    if (this.environment != null) {
       addEnvToProto();
       addEnvToProto();
     }
     }
     if (this.commands != null) {
     if (this.commands != null) {
@@ -364,37 +361,37 @@ implements ContainerLaunchContext {
   }
   }
   
   
   @Override
   @Override
-  public Map<String, String> getEnv() {
+  public Map<String, String> getEnvironment() {
     initEnv();
     initEnv();
-    return this.env;
+    return this.environment;
   }
   }
   
   
   private void initEnv() {
   private void initEnv() {
-    if (this.env != null) {
+    if (this.environment != null) {
       return;
       return;
     }
     }
     ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
     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) {
     for (StringStringMapProto c : list) {
-      this.env.put(c.getKey(), c.getValue());
+      this.environment.put(c.getKey(), c.getValue());
     }
     }
   }
   }
   
   
   @Override
   @Override
-  public void setEnv(final Map<String, String> env) {
+  public void setEnvironment(final Map<String, String> env) {
     if (env == null)
     if (env == null)
       return;
       return;
     initEnv();
     initEnv();
-    this.env.clear();
-    this.env.putAll(env);
+    this.environment.clear();
+    this.environment.putAll(env);
   }
   }
   
   
   private void addEnvToProto() {
   private void addEnvToProto() {
     maybeInitBuilder();
     maybeInitBuilder();
-    builder.clearEnv();
-    if (env == null)
+    builder.clearEnvironment();
+    if (environment == null)
       return;
       return;
     Iterable<StringStringMapProto> iterable = 
     Iterable<StringStringMapProto> iterable = 
         new Iterable<StringStringMapProto>() {
         new Iterable<StringStringMapProto>() {
@@ -403,7 +400,7 @@ implements ContainerLaunchContext {
       public Iterator<StringStringMapProto> iterator() {
       public Iterator<StringStringMapProto> iterator() {
         return new Iterator<StringStringMapProto>() {
         return new Iterator<StringStringMapProto>() {
           
           
-          Iterator<String> keyIter = env.keySet().iterator();
+          Iterator<String> keyIter = environment.keySet().iterator();
           
           
           @Override
           @Override
           public void remove() {
           public void remove() {
@@ -414,7 +411,7 @@ implements ContainerLaunchContext {
           public StringStringMapProto next() {
           public StringStringMapProto next() {
             String key = keyIter.next();
             String key = keyIter.next();
             return StringStringMapProto.newBuilder().setKey(key).setValue(
             return StringStringMapProto.newBuilder().setKey(key).setValue(
-                (env.get(key))).build();
+                (environment.get(key))).build();
           }
           }
           
           
           @Override
           @Override
@@ -424,7 +421,7 @@ implements ContainerLaunchContext {
         };
         };
       }
       }
     };
     };
-    builder.addAllEnv(iterable);
+    builder.addAllEnvironment(iterable);
   }
   }
 
 
   private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
   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 {
 message ApplicationSubmissionContextProto {
   optional ApplicationIdProto application_id = 1;
   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 {
 message YarnClusterMetricsProto {
@@ -242,7 +236,7 @@ message ContainerLaunchContextProto {
   repeated StringLocalResourceMapProto localResources = 4;
   repeated StringLocalResourceMapProto localResources = 4;
   optional bytes container_tokens = 5;
   optional bytes container_tokens = 5;
   repeated StringBytesMapProto service_data = 6;
   repeated StringBytesMapProto service_data = 6;
-  repeated StringStringMapProto env = 7;
+  repeated StringStringMapProto environment = 7;
   repeated string command = 8;
   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";
     RM_PREFIX + "max-completed-applications";
   public static final int DEFAULT_RM_MAX_COMPLETED_APPLICATIONS = 10000;
   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
   // 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 service = serviceList.get(i);
         service.start();
         service.start();
       }
       }
-    } catch(Throwable e) {
+      super.start();
+    } catch (Throwable e) {
       LOG.error("Error starting services " + getName(), 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);
       throw new YarnException("Failed to Start " + getName(), e);
     }
     }
-    super.start();
+
   }
   }
 
 
   public synchronized void stop() {
   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();
     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) {
   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) {
     if (finished > 0) {
       return finished - started;
       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) {
   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 org.apache.hadoop.yarn.webapp.WebAppException;
 
 
 import java.lang.reflect.Method;
 import java.lang.reflect.Method;
+import java.util.Map;
+
 import com.google.inject.Module;
 import com.google.inject.Module;
 import com.google.inject.Scopes;
 import com.google.inject.Scopes;
 import com.google.inject.servlet.RequestScoped;
 import com.google.inject.servlet.RequestScoped;
@@ -126,22 +128,31 @@ public class WebAppTests {
     }
     }
   }
   }
 
 
-  @SuppressWarnings("unchecked")
   public static <T> Injector testController(Class<? extends Controller> ctrlr,
   public static <T> Injector testController(Class<? extends Controller> ctrlr,
       String methodName) {
       String methodName) {
     return testController(ctrlr, methodName, null, null);
     return testController(ctrlr, methodName, null, null);
   }
   }
 
 
   public static <T> Injector testPage(Class<? extends View> page, Class<T> api,
   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 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);
     flushOutput(injector);
     return 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
   // convenience
-  @SuppressWarnings("unchecked")
   public static <T> Injector testPage(Class<? extends View> page) {
   public static <T> Injector testPage(Class<? extends View> page) {
     return testPage(page, null, null);
     return testPage(page, null, null);
   }
   }
@@ -155,7 +166,6 @@ public class WebAppTests {
   }
   }
 
 
   // convenience
   // convenience
-  @SuppressWarnings("unchecked")
   public static <T> Injector testBlock(Class<? extends SubView> block) {
   public static <T> Injector testBlock(Class<? extends SubView> block) {
     return testBlock(block, null, null);
     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);
     dispatcher.register(ContainerManagerEventType.class, containerManager);
     addService(dispatcher);
     addService(dispatcher);
 
 
-    Runtime.getRuntime().addShutdownHook(new Thread() {
-          @Override
-          public void run() {
-            NodeManager.this.stop();
-          }
-        });
-
     DefaultMetricsSystem.initialize("NodeManager");
     DefaultMetricsSystem.initialize("NodeManager");
 
 
     // StatusUpdater should be added last so that it get started last 
     // 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) {
   public static void main(String[] args) {
     StringUtils.startupShutdownMessage(NodeManager.class, args, LOG);
     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();
     final Map<Path,String> localResources = container.getLocalizedResources();
     String containerIdStr = ConverterUtils.toString(container.getContainerID());
     String containerIdStr = ConverterUtils.toString(container.getContainerID());
     final String user = launchContext.getUser();
     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();
     final List<String> command = launchContext.getCommands();
     int ret = -1;
     int ret = -1;
 
 
@@ -109,7 +109,7 @@ public class ContainerLaunch implements Callable<Integer> {
       }
       }
       launchContext.setCommands(newCmds);
       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());
       Map<String, String> newEnvs = new HashMap<String, String>(envs.size());
       for (Entry<String, String> entry : envs.entrySet()) {
       for (Entry<String, String> entry : envs.entrySet()) {
         newEnvs.put(
         newEnvs.put(
@@ -118,7 +118,7 @@ public class ContainerLaunch implements Callable<Integer> {
                 ApplicationConstants.LOG_DIR_EXPANSION_VAR,
                 ApplicationConstants.LOG_DIR_EXPANSION_VAR,
                 containerLogDir.toUri().getPath()));
                 containerLogDir.toUri().getPath()));
       }
       }
-      launchContext.setEnv(newEnvs);
+      launchContext.setEnvironment(newEnvs);
       // /////////////////////////// End of variable expansion
       // /////////////////////////// End of variable expansion
 
 
       FileContext lfs = FileContext.getLocalFSFileContext();
       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.LocalizerSecurityInfo;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security.LocalizerTokenSecretManager;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.service.CompositeService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 
-public class ResourceLocalizationService extends AbstractService
+public class ResourceLocalizationService extends CompositeService
     implements EventHandler<LocalizationEvent>, LocalizationProtocol {
     implements EventHandler<LocalizationEvent>, LocalizationProtocol {
 
 
   private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class);
   private static final Log LOG = LogFactory.getLog(ResourceLocalizationService.class);
@@ -201,9 +202,8 @@ public class ResourceLocalizationService extends AbstractService
     localizationServerAddress = NetUtils.createSocketAddr(
     localizationServerAddress = NetUtils.createSocketAddr(
       conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS));
       conf.get(YarnConfiguration.NM_LOCALIZER_ADDRESS, YarnConfiguration.DEFAULT_NM_LOCALIZER_ADDRESS));
     localizerTracker = createLocalizerTracker(conf);
     localizerTracker = createLocalizerTracker(conf);
+    addService(localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
     dispatcher.register(LocalizerEventType.class, localizerTracker);
-    cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
-        cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
     super.init(conf);
     super.init(conf);
   }
   }
 
 
@@ -214,6 +214,8 @@ public class ResourceLocalizationService extends AbstractService
 
 
   @Override
   @Override
   public void start() {
   public void start() {
+    cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
+        cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
     server = createServer();
     server = createServer();
     LOG.info("Localizer started on port " + server.getPort());
     LOG.info("Localizer started on port " + server.getPort());
     server.start();
     server.start();
@@ -247,9 +249,7 @@ public class ResourceLocalizationService extends AbstractService
     if (server != null) {
     if (server != null) {
       server.close();
       server.close();
     }
     }
-    if (localizerTracker != null) {
-      localizerTracker.stop();
-    }
+    cacheCleanup.shutdown();
     super.stop();
     super.stop();
   }
   }
 
 
@@ -403,7 +403,7 @@ public class ResourceLocalizationService extends AbstractService
   /**
   /**
    * Sub-component handling the spawning of {@link ContainerLocalizer}s
    * 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 PublicLocalizer publicLocalizer;
     private final Map<String,LocalizerRunner> privLocalizers;
     private final Map<String,LocalizerRunner> privLocalizers;
@@ -414,9 +414,15 @@ public class ResourceLocalizationService extends AbstractService
 
 
     LocalizerTracker(Configuration conf,
     LocalizerTracker(Configuration conf,
         Map<String,LocalizerRunner> privLocalizers) {
         Map<String,LocalizerRunner> privLocalizers) {
+      super(LocalizerTracker.class.getName());
       this.publicLocalizer = new PublicLocalizer(conf);
       this.publicLocalizer = new PublicLocalizer(conf);
       this.privLocalizers = privLocalizers;
       this.privLocalizers = privLocalizers;
+    }
+    
+    @Override
+    public synchronized void start() {
       publicLocalizer.start();
       publicLocalizer.start();
+      super.start();
     }
     }
 
 
     public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
     public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
@@ -435,12 +441,14 @@ public class ResourceLocalizationService extends AbstractService
         return localizer.update(status.getResources());
         return localizer.update(status.getResources());
       }
       }
     }
     }
-
+    
+    @Override
     public void stop() {
     public void stop() {
       for (LocalizerRunner localizer : privLocalizers.values()) {
       for (LocalizerRunner localizer : privLocalizers.values()) {
         localizer.interrupt();
         localizer.interrupt();
       }
       }
       publicLocalizer.interrupt();
       publicLocalizer.interrupt();
+      super.stop();
     }
     }
 
 
     @Override
     @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.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 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.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.rmnode.RMNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler;
 import org.apache.hadoop.yarn.service.AbstractService;
 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 AtomicInteger applicationCounter = new AtomicInteger(0);
   final private YarnScheduler scheduler;
   final private YarnScheduler scheduler;
   final private RMContext rmContext;
   final private RMContext rmContext;
-  private final AMLivelinessMonitor amLivelinessMonitor;
   private final RMAppManager rmAppManager;
   private final RMAppManager rmAppManager;
 
 
   private String clientServiceBindAddress;
   private String clientServiceBindAddress;
@@ -106,7 +104,6 @@ public class ClientRMService extends AbstractService implements
     super(ClientRMService.class.getName());
     super(ClientRMService.class.getName());
     this.scheduler = scheduler;
     this.scheduler = scheduler;
     this.rmContext = rmContext;
     this.rmContext = rmContext;
-    this.amLivelinessMonitor = rmContext.getAMLivelinessMonitor();
     this.rmAppManager = rmAppManager;
     this.rmAppManager = rmAppManager;
   }
   }
   
   
@@ -195,15 +192,18 @@ public class ClientRMService extends AbstractService implements
       SubmitApplicationRequest request) throws YarnRemoteException {
       SubmitApplicationRequest request) throws YarnRemoteException {
     ApplicationSubmissionContext submissionContext = request
     ApplicationSubmissionContext submissionContext = request
         .getApplicationSubmissionContext();
         .getApplicationSubmissionContext();
-    ApplicationId applicationId = null;
-    String user = null;
+    ApplicationId applicationId = submissionContext.getApplicationId();
+    String user = submissionContext.getUser();
     try {
     try {
       user = UserGroupInformation.getCurrentUser().getShortUserName();
       user = UserGroupInformation.getCurrentUser().getShortUserName();
-      applicationId = submissionContext.getApplicationId();
       if (rmContext.getRMApps().get(applicationId) != null) {
       if (rmContext.getRMApps().get(applicationId) != null) {
         throw new IOException("Application with id " + applicationId
         throw new IOException("Application with id " + applicationId
             + " is already present! Cannot add a duplicate!");
             + " is already present! Cannot add a duplicate!");
       }
       }
+      
+      // Safety 
+      submissionContext.setUser(user);
+      
       // This needs to be synchronous as the client can query 
       // This needs to be synchronous as the client can query 
       // immediately following the submission to get the application status.
       // immediately following the submission to get the application status.
       // So call handle directly and do not send an event.
       // So call handle directly and do not send an event.
@@ -226,6 +226,7 @@ public class ClientRMService extends AbstractService implements
     return response;
     return response;
   }
   }
 
 
+  @SuppressWarnings("unchecked")
   @Override
   @Override
   public FinishApplicationResponse finishApplication(
   public FinishApplicationResponse finishApplication(
       FinishApplicationRequest request) throws YarnRemoteException {
       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();
     ApplicationId applicationId = submissionContext.getApplicationId();
     RMApp application = null;
     RMApp application = null;
     try {
     try {
@@ -224,27 +226,37 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent> {
         clientTokenStr = clientToken.encodeToUrlString();
         clientTokenStr = clientToken.encodeToUrlString();
         LOG.debug("Sending client token as " + clientTokenStr);
         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()
       ApplicationStore appStore = rmContext.getApplicationsStore()
           .createApplicationStore(submissionContext.getApplicationId(),
           .createApplicationStore(submissionContext.getApplicationId(),
           submissionContext);
           submissionContext);
+      
+      // Create RMApp
       application = new RMAppImpl(applicationId, rmContext,
       application = new RMAppImpl(applicationId, rmContext,
           this.conf, submissionContext.getApplicationName(), user,
           this.conf, submissionContext.getApplicationName(), user,
           submissionContext.getQueue(), submissionContext, clientTokenStr,
           submissionContext.getQueue(), submissionContext, clientTokenStr,
-          appStore, rmContext.getAMLivelinessMonitor(), this.scheduler,
+          appStore, this.scheduler,
           this.masterService);
           this.masterService);
 
 
-      if (rmContext.getRMApps().putIfAbsent(applicationId, application) != null) {
+      if (rmContext.getRMApps().putIfAbsent(applicationId, application) != 
+          null) {
         LOG.info("Application with id " + applicationId + 
         LOG.info("Application with id " + applicationId + 
             " is already present! Cannot add a duplicate!");
             " 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.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 {
       } else {
         this.rmContext.getDispatcher().getEventHandler().handle(
         this.rmContext.getDispatcher().getEventHandler().handle(
             new RMAppEvent(applicationId, RMAppEventType.START));
             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;
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 
 
 public class RMAppManagerSubmitEvent extends RMAppManagerEvent {
 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.io.IOException;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicBoolean;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.Recoverable;
 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;
-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.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.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 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.RMAppEventType;
@@ -101,7 +100,6 @@ public class ResourceManager extends CompositeService implements Recoverable {
   private SchedulerEventDispatcher schedulerDispatcher;
   private SchedulerEventDispatcher schedulerDispatcher;
   protected RMAppManager rmAppManager;
   protected RMAppManager rmAppManager;
 
 
-  private final AtomicBoolean shutdown = new AtomicBoolean(false);
   private WebApp webApp;
   private WebApp webApp;
   private RMContext rmContext;
   private RMContext rmContext;
   private final Store store;
   private final Store store;
@@ -490,20 +488,19 @@ public class ResourceManager extends CompositeService implements Recoverable {
   
   
   public static void main(String argv[]) {
   public static void main(String argv[]) {
     StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
     StringUtils.startupShutdownMessage(ResourceManager.class, argv, LOG);
-    ResourceManager resourceManager = null;
     try {
     try {
       Configuration conf = new YarnConfiguration();
       Configuration conf = new YarnConfiguration();
       Store store =  StoreFactory.getStore(conf);
       Store store =  StoreFactory.getStore(conf);
-      resourceManager = new ResourceManager(store);
+      ResourceManager resourceManager = new ResourceManager(store);
+      Runtime.getRuntime().addShutdownHook(
+          new CompositeServiceShutdownHook(resourceManager));
       resourceManager.init(conf);
       resourceManager.init(conf);
       //resourceManager.recover(store.restore());
       //resourceManager.recover(store.restore());
       //store.doneWithRecovery();
       //store.doneWithRecovery();
       resourceManager.start();
       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.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
@@ -120,7 +119,8 @@ public class AMLauncher implements Runnable {
         + " for AM " + application.getAppAttemptId());  
         + " for AM " + application.getAppAttemptId());  
     ContainerLaunchContext launchContext =
     ContainerLaunchContext launchContext =
         createAMContainerLaunchContext(applicationContext, masterContainerID);
         createAMContainerLaunchContext(applicationContext, masterContainerID);
-    StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
+    StartContainerRequest request = 
+        recordFactory.newRecordInstance(StartContainerRequest.class);
     request.setContainerLaunchContext(launchContext);
     request.setContainerLaunchContext(launchContext);
     containerMgrProxy.startContainer(request);
     containerMgrProxy.startContainer(request);
     LOG.info("Done launching container " + application.getMasterContainer() 
     LOG.info("Done launching container " + application.getMasterContainer() 
@@ -130,7 +130,8 @@ public class AMLauncher implements Runnable {
   private void cleanup() throws IOException {
   private void cleanup() throws IOException {
     connect();
     connect();
     ContainerId containerId = application.getMasterContainer().getId();
     ContainerId containerId = application.getMasterContainer().getId();
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
+    StopContainerRequest stopRequest = 
+        recordFactory.newRecordInstance(StopContainerRequest.class);
     stopRequest.setContainerId(containerId);
     stopRequest.setContainerId(containerId);
     containerMgrProxy.stopContainer(stopRequest);
     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.
     final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
 
 
     UserGroupInformation currentUser =
     UserGroupInformation currentUser =
-        UserGroupInformation.createRemoteUser("TODO"); // TODO
+        UserGroupInformation.createRemoteUser("yarn"); // TODO
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
       ContainerToken containerToken = container.getContainerToken();
       ContainerToken containerToken = container.getContainerToken();
       Token<ContainerTokenIdentifier> token =
       Token<ContainerTokenIdentifier> token =
@@ -170,8 +171,8 @@ public class AMLauncher implements Runnable {
       ContainerId containerID) throws IOException {
       ContainerId containerID) throws IOException {
 
 
     // Construct the actual Container
     // Construct the actual Container
-    ContainerLaunchContext container = recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    container.setCommands(applicationMasterContext.getCommandList());
+    ContainerLaunchContext container = 
+        applicationMasterContext.getAMContainerSpec();
     StringBuilder mergedCommand = new StringBuilder();
     StringBuilder mergedCommand = new StringBuilder();
     String failCount = Integer.toString(application.getAppAttemptId()
     String failCount = Integer.toString(application.getAppAttemptId()
         .getAttemptId());
         .getAttemptId());
@@ -189,34 +190,28 @@ public class AMLauncher implements Runnable {
    
    
     LOG.info("Command to launch container " + 
     LOG.info("Command to launch container " + 
         containerID + " : " + mergedCommand);
         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.setContainerId(containerID);
     container.setUser(applicationMasterContext.getUser());
     container.setUser(applicationMasterContext.getUser());
-    container.setResource(applicationMasterContext.getMasterCapability());
-    container.setLocalResources(applicationMasterContext.getAllResourcesTodo());
-    container.setContainerTokens(applicationMasterContext.getFsTokensTodo());
+    setupTokensAndEnv(container);
+    
     return container;
     return container;
   }
   }
 
 
-  private Map<String, String> setupTokensInEnv(
-      ApplicationSubmissionContext asc)
+  private void setupTokensAndEnv(
+      ContainerLaunchContext container)
       throws IOException {
       throws IOException {
-    Map<String, String> env =
-      new HashMap<String, String>();
+    Map<String, String> environment = container.getEnvironment();
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
       // TODO: Security enabled/disabled info should come from RM.
       // TODO: Security enabled/disabled info should come from RM.
 
 
       Credentials credentials = new Credentials();
       Credentials credentials = new Credentials();
 
 
       DataInputByteBuffer dibb = new DataInputByteBuffer();
       DataInputByteBuffer dibb = new DataInputByteBuffer();
-      if (asc.getFsTokensTodo() != null) {
+      if (container.getContainerTokens() != null) {
         // TODO: Don't do this kind of checks everywhere.
         // TODO: Don't do this kind of checks everywhere.
-        dibb.reset(asc.getFsTokensTodo());
+        dibb.reset(container.getContainerTokens());
         credentials.readTokenStorageStream(dibb);
         credentials.readTokenStorageStream(dibb);
       }
       }
 
 
@@ -236,14 +231,16 @@ public class AMLauncher implements Runnable {
       token.setService(new Text(resolvedAddr));
       token.setService(new Text(resolvedAddr));
       String appMasterTokenEncoded = token.encodeToUrlString();
       String appMasterTokenEncoded = token.encodeToUrlString();
       LOG.debug("Putting appMaster token in env : " + appMasterTokenEncoded);
       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);
           appMasterTokenEncoded);
 
 
       // Add the RM token
       // Add the RM token
       credentials.addToken(new Text(resolvedAddr), token);
       credentials.addToken(new Text(resolvedAddr), token);
       DataOutputBuffer dob = new DataOutputBuffer();
       DataOutputBuffer dob = new DataOutputBuffer();
       credentials.writeTokenStorageToStream(dob);
       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(
       ApplicationTokenIdentifier identifier = new ApplicationTokenIdentifier(
           application.getAppAttemptId().getApplicationId());
           application.getAppAttemptId().getApplicationId());
@@ -252,9 +249,10 @@ public class AMLauncher implements Runnable {
       String encoded =
       String encoded =
           Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
           Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
       LOG.debug("The encoded client secret-key to be put in env : " + encoded);
       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")
   @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
   // Mutable fields
   private long startTime;
   private long startTime;
   private long finishTime;
   private long finishTime;
-  private AMLivelinessMonitor amLivelinessMonitor;
   private RMAppAttempt currentAttempt;
   private RMAppAttempt currentAttempt;
 
 
   private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
   private static final FinalTransition FINAL_TRANSITION = new FinalTransition();
@@ -163,7 +162,7 @@ public class RMAppImpl implements RMApp {
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
   public RMAppImpl(ApplicationId applicationId, RMContext rmContext,
       Configuration config, String name, String user, String queue,
       Configuration config, String name, String user, String queue,
       ApplicationSubmissionContext submissionContext, String clientTokenStr,
       ApplicationSubmissionContext submissionContext, String clientTokenStr,
-      ApplicationStore appStore, AMLivelinessMonitor amLivelinessMonitor,
+      ApplicationStore appStore, 
       YarnScheduler scheduler, ApplicationMasterService masterService) {
       YarnScheduler scheduler, ApplicationMasterService masterService) {
 
 
     this.applicationId = applicationId;
     this.applicationId = applicationId;
@@ -176,7 +175,6 @@ public class RMAppImpl implements RMApp {
     this.submissionContext = submissionContext;
     this.submissionContext = submissionContext;
     this.clientTokenStr = clientTokenStr;
     this.clientTokenStr = clientTokenStr;
     this.appStore = appStore;
     this.appStore = appStore;
-    this.amLivelinessMonitor = amLivelinessMonitor;
     this.scheduler = scheduler;
     this.scheduler = scheduler;
     this.masterService = masterService;
     this.masterService = masterService;
     this.startTime = System.currentTimeMillis();
     this.startTime = System.currentTimeMillis();
@@ -380,6 +378,7 @@ public class RMAppImpl implements RMApp {
     }
     }
   }
   }
 
 
+  @SuppressWarnings("unchecked")
   private void createNewAttempt() {
   private void createNewAttempt() {
     ApplicationAttemptId appAttemptId = Records
     ApplicationAttemptId appAttemptId = Records
         .newRecord(ApplicationAttemptId.class);
         .newRecord(ApplicationAttemptId.class);
@@ -434,6 +433,7 @@ public class RMAppImpl implements RMApp {
       return nodes;
       return nodes;
     }
     }
 
 
+    @SuppressWarnings("unchecked")
     public void transition(RMAppImpl app, RMAppEvent event) {
     public void transition(RMAppImpl app, RMAppEvent event) {
       Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
       Set<NodeId> nodes = getNodesOnWhichAttemptRan(app);
       for (NodeId nodeId : nodes) {
       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;
                              RMAppAttemptEvent> stateMachine;
 
 
   private final RMContext rmContext;
   private final RMContext rmContext;
+  @SuppressWarnings("rawtypes")
   private final EventHandler eventHandler;
   private final EventHandler eventHandler;
   private final YarnScheduler scheduler;
   private final YarnScheduler scheduler;
   private final ApplicationMasterService masterService;
   private final ApplicationMasterService masterService;
@@ -459,7 +460,7 @@ public class RMAppAttemptImpl implements RMAppAttempt {
       // Request a container for the AM.
       // Request a container for the AM.
       ResourceRequest request = BuilderUtils.newResourceRequest(
       ResourceRequest request = BuilderUtils.newResourceRequest(
           AM_CONTAINER_PRIORITY, "*", appAttempt.submissionContext
           AM_CONTAINER_PRIORITY, "*", appAttempt.submissionContext
-              .getMasterCapability(), 1);
+              .getAMContainerSpec().getResource(), 1);
       LOG.debug("About to request resources for AM of "
       LOG.debug("About to request resources for AM of "
           + appAttempt.applicationAttemptId + " required " + request);
           + 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 
     // Check if we need containers on this rack 
     ResourceRequest rackLocalRequest = 
     ResourceRequest rackLocalRequest = 
       application.getResourceRequest(priority, node.getRackName());
       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 (type == NodeType.RACK_LOCAL) {
-      if (rackLocalRequest == null) {
-        return false;
-      } else {
-        return rackLocalRequest.getNumContainers() > 0;      
-      }
+      return true;
     }
     }
 
 
     // Check if we need containers on this host
     // Check if we need containers on this host
     if (type == NodeType.NODE_LOCAL) {
     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...
       // Now check if we need containers on this host...
       ResourceRequest nodeLocalRequest = 
       ResourceRequest nodeLocalRequest = 
         application.getResourceRequest(priority, node.getHostName());
         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);
     return nodes.get(nodeId);
   }
   }
   
   
+  @SuppressWarnings("unchecked")
   private synchronized void addApplication(ApplicationAttemptId appAttemptId,
   private synchronized void addApplication(ApplicationAttemptId appAttemptId,
       String queueName, String user) {
       String queueName, String user) {
     // TODO: Fix store
     // TODO: Fix store
@@ -440,6 +441,14 @@ public class FifoScheduler implements ResourceScheduler {
     ResourceRequest request = 
     ResourceRequest request = 
       application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
       application.getResourceRequest(priority, node.getRMNode().getNodeAddress());
     if (request != null) {
     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 = 
       int assignableContainers = 
         Math.min(
         Math.min(
             getMaxAllocatableContainers(application, priority, node, 
             getMaxAllocatableContainers(application, priority, node, 
@@ -458,6 +467,13 @@ public class FifoScheduler implements ResourceScheduler {
     ResourceRequest request = 
     ResourceRequest request = 
       application.getResourceRequest(priority, node.getRMNode().getRackName());
       application.getResourceRequest(priority, node.getRMNode().getRackName());
     if (request != null) {
     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 = 
       int assignableContainers = 
         Math.min(
         Math.min(
             getMaxAllocatableContainers(application, priority, node, 
             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 static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
 
 
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.util.Apps;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 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.TABLE;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
@@ -59,7 +58,8 @@ class AppsBlock extends HtmlBlock {
       String appId = app.getApplicationId().toString();
       String appId = app.getApplicationId().toString();
       String trackingUrl = app.getTrackingUrl();
       String trackingUrl = app.getTrackingUrl();
       String ui = trackingUrl == null || trackingUrl.isEmpty() ? "UNASSIGNED" :
       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);
       String percent = String.format("%.1f", app.getProgress() * 100);
       tbody.
       tbody.
         tr().
         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.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 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.api.records.Resource;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.ApplicationMasterLauncher;
@@ -81,13 +82,17 @@ public class MockRM extends ResourceManager {
     ApplicationId appId = resp.getApplicationId();
     ApplicationId appId = resp.getApplicationId();
     
     
     SubmitApplicationRequest req = Records.newRecord(SubmitApplicationRequest.class);
     SubmitApplicationRequest req = Records.newRecord(SubmitApplicationRequest.class);
-    ApplicationSubmissionContext sub = Records.newRecord(ApplicationSubmissionContext.class);
+    ApplicationSubmissionContext sub = 
+        Records.newRecord(ApplicationSubmissionContext.class);
     sub.setApplicationId(appId);
     sub.setApplicationId(appId);
     sub.setApplicationName("");
     sub.setApplicationName("");
     sub.setUser("");
     sub.setUser("");
+    ContainerLaunchContext clc = 
+        Records.newRecord(ContainerLaunchContext.class);
     Resource capability = Records.newRecord(Resource.class);
     Resource capability = Records.newRecord(Resource.class);
     capability.setMemory(masterMemory);
     capability.setMemory(masterMemory);
-    sub.setMasterCapability(capability);
+    clc.setResource(capability);
+    sub.setAMContainerSpec(clc);
     req.setApplicationSubmissionContext(sub);
     req.setApplicationSubmissionContext(sub);
     
     
     client.submitApplication(req);
     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;
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 
-import static org.mockito.Mockito.*;
 
 
-import java.util.ArrayList;
 import java.util.List;
 import java.util.List;
-import java.util.LinkedList;
-import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentMap;
 
 
-
 import junit.framework.Assert;
 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.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 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.ApplicationTokenSecretManager;
 import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
 import org.apache.hadoop.yarn.security.client.ClientToAMSecretManager;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 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.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager;
 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.server.resourcemanager.scheduler.capacity.CapacityScheduler;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service;
 
 
-import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -75,7 +65,6 @@ import com.google.common.collect.Lists;
  */
  */
 
 
 public class TestAppManager{
 public class TestAppManager{
-  private static final Log LOG = LogFactory.getLog(TestAppManager.class);
   private static RMAppEventType appEventType = RMAppEventType.KILL; 
   private static RMAppEventType appEventType = RMAppEventType.KILL; 
 
 
   public synchronized RMAppEventType getAppEventType() {
   public synchronized RMAppEventType getAppEventType() {
@@ -117,10 +106,8 @@ public class TestAppManager{
   public class TestAppManagerDispatcher implements
   public class TestAppManagerDispatcher implements
       EventHandler<RMAppManagerEvent> {
       EventHandler<RMAppManagerEvent> {
 
 
-    private final RMContext rmContext;
 
 
-    public TestAppManagerDispatcher(RMContext rmContext) {
-      this.rmContext = rmContext;
+    public TestAppManagerDispatcher() {
     }
     }
 
 
     @Override
     @Override
@@ -132,15 +119,11 @@ public class TestAppManager{
   public class TestDispatcher implements
   public class TestDispatcher implements
       EventHandler<RMAppEvent> {
       EventHandler<RMAppEvent> {
 
 
-    private final RMContext rmContext;
-
-    public TestDispatcher(RMContext rmContext) {
-      this.rmContext = rmContext;
+    public TestDispatcher() {
     }
     }
 
 
     @Override
     @Override
     public void handle(RMAppEvent event) {
     public void handle(RMAppEvent event) {
-      ApplicationId appID = event.getApplicationId();
       //RMApp rmApp = this.rmContext.getRMApps().get(appID);
       //RMApp rmApp = this.rmContext.getRMApps().get(appID);
       setAppEventType(event.getType());
       setAppEventType(event.getType());
       System.out.println("in handle routine " + getAppEventType().toString());
       System.out.println("in handle routine " + getAppEventType().toString());
@@ -178,7 +161,8 @@ public class TestAppManager{
     public void setCompletedAppsMax(int max) {
     public void setCompletedAppsMax(int max) {
       super.setCompletedAppsMax(max);
       super.setCompletedAppsMax(max);
     }
     }
-    public void submitApplication(ApplicationSubmissionContext submissionContext) {
+    public void submitApplication(
+        ApplicationSubmissionContext submissionContext) {
       super.submitApplication(submissionContext);
       super.submitApplication(submissionContext);
     }
     }
   }
   }
@@ -336,8 +320,9 @@ public class TestAppManager{
   }
   }
 
 
   protected void setupDispatcher(RMContext rmContext, Configuration conf) {
   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(RMAppEventType.class, testDispatcher);
     rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher);
     rmContext.getDispatcher().register(RMAppManagerEventType.class, testAppManagerDispatcher);
     ((Service)rmContext.getDispatcher()).init(conf);
     ((Service)rmContext.getDispatcher()).init(conf);
@@ -359,7 +344,8 @@ public class TestAppManager{
 
 
     ApplicationId appID = MockApps.newAppID(1);
     ApplicationId appID = MockApps.newAppID(1);
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
     RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
+    ApplicationSubmissionContext context = 
+        recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(appID);
     context.setApplicationId(appID);
     setupDispatcher(rmContext, conf);
     setupDispatcher(rmContext, conf);
 
 
@@ -367,8 +353,12 @@ public class TestAppManager{
     RMApp app = rmContext.getRMApps().get(appID);
     RMApp app = rmContext.getRMApps().get(appID);
     Assert.assertNotNull("app is null", app);
     Assert.assertNotNull("app is null", app);
     Assert.assertEquals("app id doesn't match", appID, app.getApplicationId());
     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.assertEquals("app state doesn't match", RMAppState.NEW, app.getState());
     Assert.assertNotNull("app store is null", app.getApplicationStore());
     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,
     RMApp application = new RMAppImpl(applicationId, rmContext,
           conf, name, user,
           conf, name, user,
           queue, submissionContext, clientTokenStr,
           queue, submissionContext, clientTokenStr,
-          appStore, rmContext.getAMLivelinessMonitor(), scheduler,
+          appStore, scheduler,
           masterService);
           masterService);
 
 
     testAppStartState(applicationId, user, name, queue, application);
     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
   @Test
   public void testLocalityScheduling() throws Exception {
   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
   @After
   public void tearDown() throws Exception {
   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.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.List;
 
 
 import junit.framework.Assert;
 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.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 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.ContainerToken;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 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.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
 import org.apache.hadoop.yarn.security.SchedulerSecurityInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 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.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
@@ -137,15 +140,11 @@ public class TestContainerTokenSecretManager {
     ApplicationSubmissionContext appSubmissionContext =
     ApplicationSubmissionContext appSubmissionContext =
         recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
         recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     appSubmissionContext.setApplicationId(appID);
     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.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
     // TODO: Use a resource to work around bugs. Today NM doesn't create local
     // app-dirs if there are no file to download!!
     // app-dirs if there are no file to download!!
@@ -162,10 +161,11 @@ public class TestContainerTokenSecretManager {
     rsrc.setTimestamp(file.lastModified());
     rsrc.setTimestamp(file.lastModified());
     rsrc.setType(LocalResourceType.FILE);
     rsrc.setType(LocalResourceType.FILE);
     rsrc.setVisibility(LocalResourceVisibility.PRIVATE);
     rsrc.setVisibility(LocalResourceVisibility.PRIVATE);
-    appSubmissionContext.setResourceTodo("testFile", rsrc);
+    amContainer.setLocalResources(Collections.singletonMap("testFile", rsrc));
     SubmitApplicationRequest submitRequest = recordFactory
     SubmitApplicationRequest submitRequest = recordFactory
         .newRecordInstance(SubmitApplicationRequest.class);
         .newRecordInstance(SubmitApplicationRequest.class);
     submitRequest.setApplicationSubmissionContext(appSubmissionContext);
     submitRequest.setApplicationSubmissionContext(appSubmissionContext);
+    appSubmissionContext.setAMContainerSpec(amContainer);
     resourceManager.getClientRMService().submitApplication(submitRequest);
     resourceManager.getClientRMService().submitApplication(submitRequest);
 
 
     // Wait till container gets allocated for AM
     // Wait till container gets allocated for AM

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

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