Sfoglia il codice sorgente

Committing rest of merge from trunk (accidentally only committed the HDFS portion before)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1214546 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 anni fa
parent
commit
b77c107ccb
100 ha cambiato i file con 4063 aggiunte e 1797 eliminazioni
  1. 5 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 0 6
      hadoop-common-project/hadoop-common/pom.xml
  3. 0 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
  4. 0 269
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java
  5. 2 3
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java
  6. 1 1
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
  7. 0 42
      hadoop-common-project/hadoop-common/src/test/avro/AvroSpecificTestProtocol.avpr
  8. 0 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java
  9. 0 227
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java
  10. 7 0
      hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml
  11. 1 0
      hadoop-hdfs-project/pom.xml
  12. 26 0
      hadoop-mapreduce-project/CHANGES.txt
  13. 49 66
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
  14. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
  15. 22 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  16. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
  17. 128 106
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/JobImpl.java
  18. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
  19. 8 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  20. 5 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebApp.java
  21. 362 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AMWebServices.java
  22. 7 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AppController.java
  23. 6 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/ConfBlock.java
  24. 77 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JAXBContextResolver.java
  25. 50 144
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java
  26. 23 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java
  27. 26 18
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java
  28. 17 14
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java
  29. 70 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/AppInfo.java
  30. 46 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfEntryInfo.java
  31. 66 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfInfo.java
  32. 54 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterGroupInfo.java
  33. 44 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterInfo.java
  34. 100 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobCounterInfo.java
  35. 349 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java
  36. 63 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskAttemptCounterInfo.java
  37. 59 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskCounterInfo.java
  38. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobsInfo.java
  39. 83 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ReduceTaskAttemptInfo.java
  40. 133 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java
  41. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptsInfo.java
  42. 49 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskCounterGroupInfo.java
  43. 46 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskCounterInfo.java
  44. 122 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java
  45. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TasksInfo.java
  46. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRClientService.java
  47. 6 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRMContainerAllocator.java
  48. 0 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  49. 0 153
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/avro/MRClientProtocol.genavro
  50. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
  51. 2 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/JobReport.java
  52. 12 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/api/records/impl/pb/JobReportPBImpl.java
  53. 3 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRBuilderUtils.java
  54. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/proto/mr_protos.proto
  55. 9 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java
  56. 56 8
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobStatus.java
  57. 15 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java
  58. 55 10
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobStatus.java
  59. 4 6
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
  60. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/counters/AbstractCounters.java
  61. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryParser.java
  62. 11 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java
  63. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobQueueClient.java
  64. 18 9
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestJobMonitorAndPrint.java
  65. 1 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
  66. 19 20
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
  67. 3 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsAboutPage.java
  68. 37 131
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobBlock.java
  69. 14 22
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java
  70. 38 44
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java
  71. 4 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebApp.java
  72. 469 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsWebServices.java
  73. 78 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/JAXBContextResolver.java
  74. 96 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptInfo.java
  75. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptsInfo.java
  76. 53 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/HistoryInfo.java
  77. 295 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java
  78. 43 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobsInfo.java
  79. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
  80. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestUberAM.java
  81. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/pom.xml
  82. 0 27
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/AMRMProtocol.genavro
  83. 0 45
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ClientRMProtocol.genavro
  84. 0 37
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ContainerManager.genavro
  85. 0 109
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/yarn-types.genavro
  86. 0 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  87. 21 15
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  88. 0 80
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java
  89. 39 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/BadRequestException.java
  90. 50 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/DefaultWrapperServlet.java
  91. 115 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java
  92. 17 18
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/NotFoundException.java
  93. 43 13
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java
  94. 30 1
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java
  95. 2 2
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java
  96. 0 10
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  97. 0 40
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/avro/ResourceTracker.genavro
  98. 4 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java
  99. 98 18
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java
  100. 1 0
      hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto

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

@@ -3,6 +3,7 @@ Hadoop Change Log
 Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES
+    HADOOP-7920. Remove Avro Rpc. (suresh)
 
   NEW FEATURES
     HADOOP-7773. Add support for protocol buffer based RPC engine.
@@ -136,6 +137,10 @@ Trunk (unreleased changes)
 
     HADOOP-7913 Fix bug in ProtoBufRpcEngine  (sanjay)
 
+    HADOOP-7810. move hadoop archive to core from tools. (tucu)
+
+    HADOOP-7892. IPC logs too verbose after "RpcKind" introduction (todd)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

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

@@ -239,11 +239,6 @@
       <artifactId>avro</artifactId>
       <scope>compile</scope>
     </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-      <scope>compile</scope>
-    </dependency>
     <dependency>
       <groupId>net.sf.kosmosfs</groupId>
       <artifactId>kfs</artifactId>
@@ -282,7 +277,6 @@
             <phase>generate-test-sources</phase>
             <goals>
               <goal>schema</goal>
-              <goal>protocol</goal>
             </goals>
           </execution>
         </executions>

+ 0 - 0
hadoop-mapreduce-project/src/tools/org/apache/hadoop/fs/HarFileSystem.java → hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java


+ 0 - 269
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroRpcEngine.java

@@ -1,269 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import java.io.Closeable;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.Method;
-import java.lang.reflect.Proxy;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.HashMap;
-
-import javax.net.SocketFactory;
-
-import org.apache.avro.ipc.Responder;
-import org.apache.avro.ipc.Transceiver;
-import org.apache.avro.ipc.reflect.ReflectRequestor;
-import org.apache.avro.ipc.reflect.ReflectResponder;
-import org.apache.avro.ipc.specific.SpecificRequestor;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
-
-/** Tunnel Avro-format RPC requests over a Hadoop {@link RPC} connection.  This
- * does not give cross-language wire compatibility, since the Hadoop RPC wire
- * format is non-standard, but it does permit use of Avro's protocol versioning
- * features for inter-Java RPCs. */
-@InterfaceStability.Evolving
-public class AvroRpcEngine implements RpcEngine {
-  private static final Log LOG = LogFactory.getLog(RPC.class);
-
-  private static int VERSION = 1;
-
-  // the implementation we tunnel through
-  private static final RpcEngine ENGINE = new WritableRpcEngine();
-
-  /** Tunnel an Avro RPC request and response through Hadoop's RPC. */
-  private static interface TunnelProtocol extends VersionedProtocol {
-    //WritableRpcEngine expects a versionID in every protocol.
-    public static final long versionID = VERSION;
-    /** All Avro methods and responses go through this. */
-    BufferListWritable call(String protocol, BufferListWritable request)
-      throws IOException;
-  }
-
-  /** A Writable that holds a List<ByteBuffer>, The Avro RPC Transceiver's
-   * basic unit of data transfer.*/
-  private static class BufferListWritable implements Writable {
-    private List<ByteBuffer> buffers;
-
-    public BufferListWritable() {}                // required for RPC Writables
-
-    public BufferListWritable(List<ByteBuffer> buffers) {
-      this.buffers = buffers;
-    }
-
-    public void readFields(DataInput in) throws IOException {
-      int size = in.readInt();
-      buffers = new ArrayList<ByteBuffer>(size);
-      for (int i = 0; i < size; i++) {
-        int length = in.readInt();
-        ByteBuffer buffer = ByteBuffer.allocate(length);
-        in.readFully(buffer.array(), 0, length);
-        buffers.add(buffer);
-      }
-    }
-  
-    public void write(DataOutput out) throws IOException {
-      out.writeInt(buffers.size());
-      for (ByteBuffer buffer : buffers) {
-        out.writeInt(buffer.remaining());
-        out.write(buffer.array(), buffer.position(), buffer.remaining());
-      }
-    }
-  }
-
-  /** An Avro RPC Transceiver that tunnels client requests through Hadoop
-   * RPC. */
-  private static class ClientTransceiver extends Transceiver {
-    private TunnelProtocol tunnel;
-    private InetSocketAddress remote;
-    private String protocol;
-  
-    public ClientTransceiver(InetSocketAddress addr,
-                             UserGroupInformation ticket,
-                             Configuration conf, SocketFactory factory,
-                             int rpcTimeout, String protocol)
-      throws IOException {
-      this.tunnel = ENGINE.getProxy(TunnelProtocol.class, VERSION,
-                                        addr, ticket, conf, factory,
-                                        rpcTimeout).getProxy();
-      this.remote = addr;
-      this.protocol = protocol;
-    }
-
-    public String getRemoteName() { return remote.toString(); }
-
-    public List<ByteBuffer> transceive(List<ByteBuffer> request)
-      throws IOException {
-      return tunnel.call(protocol, new BufferListWritable(request)).buffers;
-    }
-
-    public List<ByteBuffer> readBuffers() throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    public void writeBuffers(List<ByteBuffer> buffers) throws IOException {
-      throw new UnsupportedOperationException();
-    }
-
-    public void close() throws IOException {
-      RPC.stopProxy(tunnel);
-    }
-  }
-
-  /** Construct a client-side proxy object that implements the named protocol,
-   * talking to a server at the named address. 
-   * @param <T>*/
-  @SuppressWarnings("unchecked")
-  public <T> ProtocolProxy<T> getProxy(Class<T> protocol, long clientVersion,
-                         InetSocketAddress addr, UserGroupInformation ticket,
-                         Configuration conf, SocketFactory factory,
-                         int rpcTimeout)
-    throws IOException {
-    return new ProtocolProxy<T>(protocol,
-       (T)Proxy.newProxyInstance(
-         protocol.getClassLoader(),
-         new Class[] { protocol },
-         new Invoker(protocol, addr, ticket, conf, factory, rpcTimeout)),
-       false);
-  }
-
-  private class Invoker implements InvocationHandler, Closeable {
-    private final ClientTransceiver tx;
-    private final SpecificRequestor requestor;
-    public Invoker(Class<?> protocol, InetSocketAddress addr,
-                   UserGroupInformation ticket, Configuration conf,
-                   SocketFactory factory,
-                   int rpcTimeout) throws IOException {
-      this.tx = new ClientTransceiver(addr, ticket, conf, factory, rpcTimeout,
-                                      protocol.getName());
-      this.requestor = createRequestor(protocol, tx);
-    }
-    @Override public Object invoke(Object proxy, Method method, Object[] args) 
-      throws Throwable {
-      return requestor.invoke(proxy, method, args);
-    }
-    public void close() throws IOException {
-      tx.close();
-    }
-  }
-
-  protected SpecificRequestor createRequestor(Class<?> protocol, 
-      Transceiver transeiver) throws IOException {
-    return new ReflectRequestor(protocol, transeiver);
-  }
-
-  protected Responder createResponder(Class<?> iface, Object impl) {
-    return new ReflectResponder(iface, impl);
-  }
-
-  /** An Avro RPC Responder that can process requests passed via Hadoop RPC. */
-  private class TunnelResponder implements TunnelProtocol {
-    private Map<String, Responder> responders =
-      new HashMap<String, Responder>();
-
-    public void addProtocol(Class<?> iface, Object impl) {
-      responders.put(iface.getName(), createResponder(iface, impl));
-    }
-
-    @Override
-    public long getProtocolVersion(String protocol, long version)
-    throws IOException {
-      return VERSION;
-    }
-
-    @Override
-    public ProtocolSignature getProtocolSignature(
-        String protocol, long version, int clientMethodsHashCode)
-      throws IOException {
-      return ProtocolSignature.getProtocolSignature
-        (clientMethodsHashCode, VERSION, TunnelProtocol.class);
-    }
-
-    public BufferListWritable call(String protocol, BufferListWritable request)
-      throws IOException {
-      Responder responder = responders.get(protocol);
-      if (responder == null)
-        throw new IOException("No responder for: "+protocol);
-      return new BufferListWritable(responder.respond(request.buffers));
-    }
-
-  }
-
-  public Object[] call(Method method, Object[][] params,
-                       InetSocketAddress[] addrs, UserGroupInformation ticket,
-                       Configuration conf) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  private class Server extends WritableRpcEngine.Server {
-    private TunnelResponder responder = new TunnelResponder();
-
-    public Server(Class<?> iface, Object impl, String bindAddress,
-                  int port, int numHandlers, int numReaders,
-                  int queueSizePerHandler, boolean verbose,
-                  Configuration conf, 
-                  SecretManager<? extends TokenIdentifier> secretManager
-                  ) throws IOException {
-      super((Class)null, new Object(), conf,
-            bindAddress, port, numHandlers, numReaders,
-            queueSizePerHandler, verbose, secretManager);
-      // RpcKind is WRITABLE since Avro is tunneled through WRITABLE
-      super.addProtocol(RpcKind.RPC_WRITABLE, TunnelProtocol.class, responder);
-      responder.addProtocol(iface, impl);
-    }
-
-
-    @Override
-    public Server
-      addProtocol(RpcKind rpcKind, Class<?> protocolClass, Object protocolImpl)
-        throws IOException {
-      responder.addProtocol(protocolClass, protocolImpl);
-      return this;
-    }
-  }
-
-  /** Construct a server for a protocol implementation instance listening on a
-   * port and address. */
-  public RPC.Server getServer(Class<?> iface, Object impl, String bindAddress,
-                              int port, int numHandlers, int numReaders,
-                              int queueSizePerHandler, boolean verbose,
-                              Configuration conf, 
-                       SecretManager<? extends TokenIdentifier> secretManager
-                              ) throws IOException {
-    return new Server
-      (iface, impl, bindAddress, port, numHandlers, numReaders,
-       queueSizePerHandler, verbose, conf, secretManager);
-  }
-
-}

+ 2 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcPayloadHeader.java

@@ -56,9 +56,8 @@ public class RpcPayloadHeader implements Writable {
   public enum RpcKind {
     RPC_BUILTIN ((short) 1),         // Used for built in calls by tests
     RPC_WRITABLE ((short) 2),        // Use WritableRpcEngine 
-    RPC_PROTOCOL_BUFFER ((short) 3), // Use ProtobufRpcEngine
-    RPC_AVRO ((short) 4);            // Use AvroRpcEngine 
-    static final short MAX_INDEX = RPC_AVRO.value; // used for array size
+    RPC_PROTOCOL_BUFFER ((short) 3); // Use ProtobufRpcEngine
+    final static short MAX_INDEX = RPC_PROTOCOL_BUFFER.value; // used for array size
     private static final short FIRST_INDEX = RPC_BUILTIN.value;    
     private final short value;
 

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

@@ -170,7 +170,7 @@ public abstract class Server {
       throw new IllegalArgumentException("ReRegistration of rpcKind: " +
           rpcKind);      
     }
-    LOG.info("rpcKind=" + rpcKind + 
+    LOG.debug("rpcKind=" + rpcKind + 
         ", rpcRequestWrapperClass=" + rpcRequestWrapperClass + 
         ", rpcInvoker=" + rpcInvoker);
   }

+ 0 - 42
hadoop-common-project/hadoop-common/src/test/avro/AvroSpecificTestProtocol.avpr

@@ -1,42 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-{
-  "protocol" : "AvroSpecificTestProtocol",
-  "namespace" : "org.apache.hadoop.ipc",
-
-  "messages" : {
-    "echo" : {
-      "request" : [ {
-        "name" : "message",
-        "type" : "string"
-      } ],
-      "response" : "string"
-    },
-    
-    "add" : {
-      "request" : [ {
-        "name" : "arg1",
-        "type" : "int"
-      }, {
-        "name" : "arg2",
-        "type" : "int",
-        "default" : 0
-      } ],
-      "response" : "int"
-    }
-  }
-}

+ 0 - 0
hadoop-mapreduce-project/src/test/mapred/org/apache/hadoop/fs/TestHarFileSystem.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystem.java


+ 0 - 227
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestAvroRpc.java

@@ -1,227 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.ipc;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import javax.security.sasl.Sasl;
-
-import junit.framework.Assert;
-import junit.framework.TestCase;
-
-import org.apache.avro.AvroRemoteException;
-import org.apache.avro.util.Utf8;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.RpcPayloadHeader.RpcKind;
-import org.apache.hadoop.ipc.TestSaslRPC.CustomSecurityInfo;
-import org.apache.hadoop.ipc.TestSaslRPC.TestTokenIdentifier;
-import org.apache.hadoop.ipc.TestSaslRPC.TestTokenSecretManager;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.SaslRpcServer;
-import org.apache.hadoop.security.SecurityInfo;
-import org.apache.hadoop.security.SecurityUtil;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
-
-/** Unit tests for AvroRpc. */
-public class TestAvroRpc extends TestCase {
-  private static final String ADDRESS = "0.0.0.0";
-
-  public static final Log LOG =
-    LogFactory.getLog(TestAvroRpc.class);
-  
-  int datasize = 1024*100;
-  int numThreads = 50;
-
-  public TestAvroRpc(String name) { super(name); }
-	
-  public static interface EmptyProtocol {}
-  public static class EmptyImpl implements EmptyProtocol {}
-
-  public static class TestImpl implements AvroTestProtocol {
-
-    public void ping() {}
-    
-    public String echo(String value) { return value; }
-
-    public int add(int v1, int v2) { return v1 + v2; }
-
-    public int error() throws Problem {
-      throw new Problem();
-    }
-  }
-
-  public void testReflect() throws Exception {
-    testReflect(false);
-  }
-
-  public void testSecureReflect() throws Exception {
-    testReflect(true);
-  }
-
-  public void testSpecific() throws Exception {
-    testSpecific(false);
-  }
-
-  public void testSecureSpecific() throws Exception {
-    testSpecific(true);
-  }
-
-  private void testReflect(boolean secure) throws Exception {
-    Configuration conf = new Configuration();
-    TestTokenSecretManager sm = null;
-    if (secure) {
-      makeSecure(conf);
-      sm = new TestTokenSecretManager();
-    }
-    UserGroupInformation.setConfiguration(conf);
-    RPC.setProtocolEngine(conf, EmptyProtocol.class, AvroRpcEngine.class);
-    RPC.setProtocolEngine(conf, AvroTestProtocol.class, AvroRpcEngine.class);
-    RPC.Server server = RPC.getServer(EmptyProtocol.class, new EmptyImpl(),
-                                      ADDRESS, 0, 5, true, conf, sm);
-    server.addProtocol(RpcKind.RPC_WRITABLE, 
-        AvroTestProtocol.class, new TestImpl());
-
-    try {
-      server.start();
-      InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
-      if (secure) {
-        addToken(sm, addr);
-        //QOP must be auth
-        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
-      }
-
-      AvroTestProtocol proxy =
-        (AvroTestProtocol)RPC.getProxy(AvroTestProtocol.class, 0, addr, conf);
-
-      proxy.ping();
-
-      String echo = proxy.echo("hello world");
-      assertEquals("hello world", echo);
-
-      int intResult = proxy.add(1, 2);
-      assertEquals(3, intResult);
-
-      boolean caught = false;
-      try {
-        proxy.error();
-      } catch (AvroRemoteException e) {
-        if(LOG.isDebugEnabled()) {
-          LOG.debug("Caught " + e);
-        }
-        caught = true;
-      }
-      assertTrue(caught);
-
-    } finally {
-      resetSecurity();
-      server.stop();
-    }
-  }
-
-  private void makeSecure(Configuration conf) {
-    conf.set(HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    conf.set("hadoop.rpc.socket.factory.class.default", "");
-    //Avro doesn't work with security annotations on protocol.
-    //Avro works ONLY with custom security context
-    SecurityUtil.setSecurityInfoProviders(new CustomSecurityInfo());
-  }
-  
-  private void resetSecurity() {
-    SecurityUtil.setSecurityInfoProviders(new SecurityInfo[0]);
-  }
-
-  private void addToken(TestTokenSecretManager sm, 
-      InetSocketAddress addr) throws IOException {
-    final UserGroupInformation current = UserGroupInformation.getCurrentUser();
-    
-    TestTokenIdentifier tokenId = new TestTokenIdentifier(new Text(current
-        .getUserName()));
-    Token<TestTokenIdentifier> token = new Token<TestTokenIdentifier>(tokenId,
-        sm);
-    Text host = new Text(addr.getAddress().getHostAddress() + ":"
-        + addr.getPort());
-    token.setService(host);
-    LOG.info("Service IP address for token is " + host);
-    current.addToken(token);
-  }
-
-  private void testSpecific(boolean secure) throws Exception {
-    Configuration conf = new Configuration();
-    TestTokenSecretManager sm = null;
-    if (secure) {
-      makeSecure(conf);
-      sm = new TestTokenSecretManager();
-    }
-    UserGroupInformation.setConfiguration(conf);
-    RPC.setProtocolEngine(conf, AvroSpecificTestProtocol.class, 
-        AvroSpecificRpcEngine.class);
-    Server server = RPC.getServer(AvroSpecificTestProtocol.class,
-        new AvroSpecificTestProtocolImpl(), ADDRESS, 0, 5, true, 
-        conf, sm);
-    try {
-      server.start();
-      InetSocketAddress addr = NetUtils.getConnectAddress(server);
-
-      if (secure) {
-        addToken(sm, addr);
-        //QOP must be auth
-        Assert.assertEquals("auth", SaslRpcServer.SASL_PROPS.get(Sasl.QOP));
-      }
-
-      AvroSpecificTestProtocol proxy =
-        (AvroSpecificTestProtocol)RPC.getProxy(AvroSpecificTestProtocol.class, 
-            0, addr, conf);
-      
-      CharSequence echo = proxy.echo("hello world");
-      assertEquals("hello world", echo.toString());
-
-      int intResult = proxy.add(1, 2);
-      assertEquals(3, intResult);
-
-    } finally {
-      resetSecurity();
-      server.stop();
-    }
-  }
-  
-  public static class AvroSpecificTestProtocolImpl implements 
-      AvroSpecificTestProtocol {
-
-    @Override
-    public int add(int arg1, int arg2) throws AvroRemoteException {
-      return arg1 + arg2;
-    }
-
-    @Override
-    public CharSequence echo(CharSequence msg) throws AvroRemoteException {
-      return msg;
-    }
-    
-  }
-
-}

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/pom.xml

@@ -269,6 +269,13 @@
     </resources>
 
     <plugins>
+      <plugin>
+	<!-- workaround for filtered/unfiltered resources in same directory -->
+	<!-- remove when maven-eclipse-plugin 2.9 is available -->
+	<groupId>org.apache.maven.plugins</groupId>
+	<artifactId>maven-eclipse-plugin</artifactId>
+	<version>2.6</version>
+      </plugin>
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>

+ 1 - 0
hadoop-hdfs-project/pom.xml

@@ -30,6 +30,7 @@
   <modules>
     <module>hadoop-hdfs</module>
     <module>hadoop-hdfs-httpfs</module>
+    <module>hadoop-hdfs/src/contrib/bkjournal</module>
   </modules>
 
   <build>

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

@@ -3,6 +3,7 @@ Hadoop MapReduce Change Log
 Trunk (unreleased changes)
 
   INCOMPATIBLE CHANGES
+    MAPREDUCE-3545. Remove Avro RPC. (suresh)
 
   NEW FEATURES
 
@@ -77,6 +78,12 @@ Trunk (unreleased changes)
     MAPREDUCE-3389. MRApps loads the 'mrapp-generated-classpath' file with 
     classpath from the build machine. (tucu)
 
+    MAPREDUCE-3544. gridmix build is broken, requires hadoop-archives to be added as 
+    ivy dependency. (tucu)
+
+    MAPREDUCE-3557. MR1 test fail to compile because of missing hadoop-archives dependency. 
+    (tucu)
+
 Release 0.23.1 - Unreleased
 
   INCOMPATIBLE CHANGES
@@ -85,6 +92,9 @@ Release 0.23.1 - Unreleased
    
    MAPREDUCE-3121. NodeManager should handle disk-failures (Ravi Gummadi via mahadev)
 
+   MAPREDUCE-2863. Support web services for YARN and MR components. (Thomas
+   Graves via vinodkv)
+
   IMPROVEMENTS
 
     MAPREDUCE-3297. Moved log related components into yarn-common so that
@@ -276,6 +286,22 @@ Release 0.23.1 - Unreleased
     MAPREDUCE-3537. Fix race condition in DefaultContainerExecutor which led
     to container localization occuring in wrong directories. (acmurthy) 
 
+    MAPREDUCE-3542. Support "FileSystemCounter" legacy counter group name for
+    compatibility. (tomwhite)
+
+    MAPREDUCE-3426. Fixed MR AM in uber mode to write map intermediate outputs
+    in the correct directory to work properly in secure mode. (Hitesh Shah via
+    vinodkv)
+
+    MAPREDUCE-3541. Fix broken TestJobQueueClient test. (Ravi Prakash via 
+    mahadev)
+
+    MAPREDUCE-3398. Fixed log aggregation to work correctly in secure mode.
+    (Siddharth Seth via vinodkv)
+
+    MAPREDUCE-3530. Fixed an NPE occuring during scheduling in the
+    ResourceManager. (Arun C Murthy via vinodkv)
+
 Release 0.23.0 - 2011-11-01 
 
   INCOMPATIBLE CHANGES

+ 49 - 66
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java

@@ -22,20 +22,19 @@ import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.io.IOException;
 import java.io.PrintStream;
-import java.net.URI;
 import java.util.HashSet;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FSError;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.TypeConverter;
@@ -47,13 +46,12 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptContainerLaunched
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 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.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherEvent;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerRemoteLaunchEvent;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.service.AbstractService;
 
 /**
@@ -80,7 +78,10 @@ public class LocalContainerLauncher extends AbstractService implements
     super(LocalContainerLauncher.class.getName());
     this.context = context;
     this.umbilical = umbilical;
-        // umbilical:  MRAppMaster creates (taskAttemptListener), passes to us  (TODO/FIXME:  pointless to use RPC to talk to self; should create LocalTaskAttemptListener or similar:  implement umbilical protocol but skip RPC stuff)
+        // umbilical:  MRAppMaster creates (taskAttemptListener), passes to us
+        // (TODO/FIXME:  pointless to use RPC to talk to self; should create
+        // LocalTaskAttemptListener or similar:  implement umbilical protocol
+        // but skip RPC stuff)
 
     try {
       curFC = FileContext.getFileContext(curDir.toURI());
@@ -152,7 +153,6 @@ public class LocalContainerLauncher extends AbstractService implements
    *     ]]
    *   - runs Task (runSubMap() or runSubReduce())
    *     - TA can safely send TA_UPDATE since in RUNNING state
-   *       [modulo possible TA-state-machine race noted below:  CHECK (TODO)]
    */
   private class SubtaskRunner implements Runnable {
 
@@ -162,6 +162,7 @@ public class LocalContainerLauncher extends AbstractService implements
     SubtaskRunner() {
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void run() {
       ContainerLauncherEvent event = null;
@@ -183,7 +184,7 @@ public class LocalContainerLauncher extends AbstractService implements
 
           ContainerRemoteLaunchEvent launchEv =
               (ContainerRemoteLaunchEvent)event;
-          TaskAttemptId attemptID = launchEv.getTaskAttemptID(); //FIXME:  can attemptID ever be null?  (only if retrieved over umbilical?)
+          TaskAttemptId attemptID = launchEv.getTaskAttemptID(); 
 
           Job job = context.getAllJobs().get(attemptID.getTaskId().getJobId());
           int numMapTasks = job.getTotalMaps();
@@ -204,7 +205,6 @@ public class LocalContainerLauncher extends AbstractService implements
           // port number is set to -1 in this case.
           context.getEventHandler().handle(
               new TaskAttemptContainerLaunchedEvent(attemptID, -1));
-          //FIXME:  race condition here?  or do we have same kind of lock on TA handler => MapTask can't send TA_UPDATE before TA_CONTAINER_LAUNCHED moves TA to RUNNING state?  (probably latter)
 
           if (numMapTasks == 0) {
             doneWithMaps = true;
@@ -259,6 +259,7 @@ public class LocalContainerLauncher extends AbstractService implements
       }
     }
 
+    @SuppressWarnings("deprecation")
     private void runSubtask(org.apache.hadoop.mapred.Task task,
                             final TaskType taskType,
                             TaskAttemptId attemptID,
@@ -270,6 +271,19 @@ public class LocalContainerLauncher extends AbstractService implements
 
       try {
         JobConf conf = new JobConf(getConfig());
+        conf.set(JobContext.TASK_ID, task.getTaskID().toString());
+        conf.set(JobContext.TASK_ATTEMPT_ID, classicAttemptID.toString());
+        conf.setBoolean(JobContext.TASK_ISMAP, (taskType == TaskType.MAP));
+        conf.setInt(JobContext.TASK_PARTITION, task.getPartition());
+        conf.set(JobContext.ID, task.getJobID().toString());
+
+        // Use the AM's local dir env to generate the intermediate step 
+        // output files
+        String[] localSysDirs = StringUtils.getTrimmedStrings(
+            System.getenv(ApplicationConstants.LOCAL_DIR_ENV));
+        conf.setStrings(MRConfig.LOCAL_DIR, localSysDirs);
+        LOG.info(MRConfig.LOCAL_DIR + " for uber task: "
+            + conf.get(MRConfig.LOCAL_DIR));
 
         // mark this as an uberized subtask so it can set task counter
         // (longer-term/FIXME:  could redefine as job counter and send
@@ -285,12 +299,12 @@ public class LocalContainerLauncher extends AbstractService implements
           if (doneWithMaps) {
             LOG.error("CONTAINER_REMOTE_LAUNCH contains a map task ("
                       + attemptID + "), but should be finished with maps");
-            // throw new RuntimeException()  (FIXME: what's appropriate here?)
+            throw new RuntimeException();
           }
 
           MapTask map = (MapTask)task;
+          map.setConf(conf);
 
-          //CODE-REVIEWER QUESTION: why not task.getConf() or map.getConf() instead of conf? do we need Task's localizeConfiguration() run on this first?
           map.run(conf, umbilical);
 
           if (renameOutputs) {
@@ -305,19 +319,23 @@ public class LocalContainerLauncher extends AbstractService implements
         } else /* TaskType.REDUCE */ {
 
           if (!doneWithMaps) {
-            //check if event-queue empty?  whole idea of counting maps vs. checking event queue is a tad wacky...but could enforce ordering (assuming no "lost events") at LocalMRAppMaster [CURRENT BUG(?):  doesn't send reduce event until maps all done]
+            // check if event-queue empty?  whole idea of counting maps vs. 
+            // checking event queue is a tad wacky...but could enforce ordering
+            // (assuming no "lost events") at LocalMRAppMaster [CURRENT BUG(?): 
+            // doesn't send reduce event until maps all done]
             LOG.error("CONTAINER_REMOTE_LAUNCH contains a reduce task ("
                       + attemptID + "), but not yet finished with maps");
-            // throw new RuntimeException()  (FIXME) // or push reduce event back onto end of queue? (probably former)
+            throw new RuntimeException();
           }
 
-          ReduceTask reduce = (ReduceTask)task;
-
           // a.k.a. "mapreduce.jobtracker.address" in LocalJobRunner:
           // set framework name to local to make task local
           conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.LOCAL_FRAMEWORK_NAME);
           conf.set(MRConfig.MASTER_ADDRESS, "local");  // bypass shuffle
 
+          ReduceTask reduce = (ReduceTask)task;
+          reduce.setConf(conf);          
+
           reduce.run(conf, umbilical);
           //relocalize();  // needed only if more than one reducer supported (is MAPREDUCE-434 fixed yet?)
         }
@@ -334,18 +352,7 @@ public class LocalContainerLauncher extends AbstractService implements
         try {
           if (task != null) {
             // do cleanup for the task
-//          if (childUGI == null) { // no need to job into doAs block
-              task.taskCleanup(umbilical);
-//          } else {
-//            final Task taskFinal = task;
-//            childUGI.doAs(new PrivilegedExceptionAction<Object>() {
-//              @Override
-//              public Object run() throws Exception {
-//                taskFinal.taskCleanup(umbilical);
-//                return null;
-//              }
-//            });
-//          }
+            task.taskCleanup(umbilical);
           }
         } catch (Exception e) {
           LOG.info("Exception cleaning up: "
@@ -354,51 +361,21 @@ public class LocalContainerLauncher extends AbstractService implements
         // Report back any failures, for diagnostic purposes
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         exception.printStackTrace(new PrintStream(baos));
-//      if (classicAttemptID != null) {
-          umbilical.reportDiagnosticInfo(classicAttemptID, baos.toString());
-//      }
+        umbilical.reportDiagnosticInfo(classicAttemptID, baos.toString());
         throw new RuntimeException();
 
       } catch (Throwable throwable) {
         LOG.fatal("Error running local (uberized) 'child' : "
             + StringUtils.stringifyException(throwable));
-//      if (classicAttemptID != null) {
-          Throwable tCause = throwable.getCause();
-          String cause = (tCause == null)
-              ? throwable.getMessage()
-              : StringUtils.stringifyException(tCause);
-          umbilical.fatalError(classicAttemptID, cause);
-//      }
+        Throwable tCause = throwable.getCause();
+        String cause = (tCause == null)
+            ? throwable.getMessage()
+                : StringUtils.stringifyException(tCause);
+            umbilical.fatalError(classicAttemptID, cause);
         throw new RuntimeException();
-
-      } finally {
-/*
-FIXME:  do we need to do any of this stuff?  (guessing not since not in own JVM)
-        RPC.stopProxy(umbilical);
-        DefaultMetricsSystem.shutdown();
-        // Shutting down log4j of the child-vm...
-        // This assumes that on return from Task.run()
-        // there is no more logging done.
-        LogManager.shutdown();
- */
       }
     }
 
-
-/* FIXME:  may not need renameMapOutputForReduce() anymore?  TEST!
-
-${local.dir}/usercache/$user/appcache/$appId/$contId/ == $cwd for containers;
-contains launch_container.sh script, which, when executed, creates symlinks and 
-sets up env
- "$local.dir"/usercache/$user/appcache/$appId/$contId/file.out
- "$local.dir"/usercache/$user/appcache/$appId/$contId/file.out.idx (?)
- "$local.dir"/usercache/$user/appcache/$appId/output/$taskId/ is where file.out* is moved after MapTask done
-
-	OHO!  no further need for this at all?  $taskId is unique per subtask
-	now => should work fine to leave alone.  TODO:  test with teragen or
-	similar
- */
-
     /**
      * Within the _local_ filesystem (not HDFS), all activity takes place within
      * a single subdir (${local.dir}/usercache/$user/appcache/$appId/$contId/),
@@ -409,14 +386,21 @@ sets up env
      * filenames instead of "file.out". (All of this is entirely internal,
      * so there are no particular compatibility issues.)
      */
+    @SuppressWarnings("deprecation")
     private void renameMapOutputForReduce(JobConf conf, TaskAttemptId mapId,
                                           MapOutputFile subMapOutputFile)
     throws IOException {
       FileSystem localFs = FileSystem.getLocal(conf);
       // move map output to reduce input
       Path mapOut = subMapOutputFile.getOutputFile();
+      FileStatus mStatus = localFs.getFileStatus(mapOut);      
       Path reduceIn = subMapOutputFile.getInputFileForWrite(
-          TypeConverter.fromYarn(mapId).getTaskID(), localFs.getLength(mapOut));
+          TypeConverter.fromYarn(mapId).getTaskID(), mStatus.getLen());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Renaming map output file for task attempt "
+            + mapId.toString() + " from original location " + mapOut.toString()
+            + " to destination " + reduceIn.toString());
+      }
       if (!localFs.mkdirs(reduceIn.getParent())) {
         throw new IOException("Mkdirs failed to create "
             + reduceIn.getParent().toString());
@@ -429,8 +413,7 @@ sets up env
      * Also within the local filesystem, we need to restore the initial state
      * of the directory as much as possible.  Compare current contents against
      * the saved original state and nuke everything that doesn't belong, with
-     * the exception of the renamed map outputs (see above).
-FIXME:  do we really need to worry about renamed map outputs, or already moved to output dir on commit?  if latter, fix comment
+     * the exception of the renamed map outputs.
      *
      * Any jobs that go out of their way to rename or delete things from the
      * local directory are considered broken and deserve what they get...

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java

@@ -236,6 +236,13 @@ public class MapReduceChildJVM {
                 getTaskLogFile(TaskLog.LogName.PROFILE)
                 )
             );
+        if (task.isMapTask()) {
+          vargs.add(conf.get(MRJobConfig.TASK_MAP_PROFILE_PARAMS, ""));
+        }
+        else {
+          vargs.add(conf.get(MRJobConfig.TASK_REDUCE_PROFILE_PARAMS, ""));
+        }
+        
       }
     }
 

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

@@ -156,6 +156,7 @@ public class MRAppMaster extends CompositeService {
   private OutputCommitter committer;
   private JobEventDispatcher jobEventDispatcher;
   private boolean inRecovery = false;
+  private SpeculatorEventDispatcher speculatorEventDispatcher;
 
   private Job job;
   private Credentials fsTokens = new Credentials(); // Filled during init
@@ -265,8 +266,9 @@ public class MRAppMaster extends CompositeService {
       addIfService(speculator);
     }
 
+    speculatorEventDispatcher = new SpeculatorEventDispatcher(conf);
     dispatcher.register(Speculator.EventType.class,
-        new SpeculatorEventDispatcher(conf));
+        speculatorEventDispatcher);
 
     // service to allocate containers from RM (if non-uber) or to fake it (uber)
     containerAllocator = createContainerAllocator(clientService, context);
@@ -386,7 +388,7 @@ public class MRAppMaster extends CompositeService {
         // This will also send the final report to the ResourceManager
         LOG.info("Calling stop for all the services");
         stop();
-        
+
         // Send job-end notification
         try {
           LOG.info("Job end notification started for jobID : "
@@ -401,14 +403,14 @@ public class MRAppMaster extends CompositeService {
       } catch (Throwable t) {
         LOG.warn("Graceful stop failed ", t);
       }
-      
+
       // Cleanup staging directory
       try {
         cleanupStagingDir();
       } catch(IOException io) {
         LOG.warn("Failed to delete staging dir");
       }
-      
+
       //Bring the process down by force.
       //Not needed after HADOOP-7140
       LOG.info("Exiting MR AppMaster..GoodBye!");
@@ -790,10 +792,6 @@ public class MRAppMaster extends CompositeService {
     // job-init to be done completely here.
     jobEventDispatcher.handle(initJobEvent);
 
-    // send init to speculator. This won't yest start as dispatcher isn't
-    // started yet.
-    dispatcher.getEventHandler().handle(
-        new SpeculatorEvent(job.getID(), clock.getTime()));
 
     // JobImpl's InitTransition is done (call above is synchronous), so the
     // "uber-decision" (MR-1220) has been made.  Query job and switch to
@@ -801,9 +799,15 @@ public class MRAppMaster extends CompositeService {
     // and container-launcher services/event-handlers).
 
     if (job.isUber()) {
+      speculatorEventDispatcher.disableSpeculation();
       LOG.info("MRAppMaster uberizing job " + job.getID()
-               + " in local container (\"uber-AM\").");
+               + " in local container (\"uber-AM\") on node "
+               + nmHost + ":" + nmPort + ".");
     } else {
+      // send init to speculator only for non-uber jobs. 
+      // This won't yet start as dispatcher isn't started yet.
+      dispatcher.getEventHandler().handle(
+          new SpeculatorEvent(job.getID(), clock.getTime()));
       LOG.info("MRAppMaster launching normal, non-uberized, multi-container "
                + "job " + job.getID() + ".");
     }
@@ -865,17 +869,24 @@ public class MRAppMaster extends CompositeService {
   private class SpeculatorEventDispatcher implements
       EventHandler<SpeculatorEvent> {
     private final Configuration conf;
+    private volatile boolean disabled;
     public SpeculatorEventDispatcher(Configuration config) {
       this.conf = config;
     }
     @Override
     public void handle(SpeculatorEvent event) {
-      if (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
-          || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false)) {
+      if (!disabled && 
+          (conf.getBoolean(MRJobConfig.MAP_SPECULATIVE, false)
+          || conf.getBoolean(MRJobConfig.REDUCE_SPECULATIVE, false))) {
         // Speculator IS enabled, direct the event to there.
         speculator.handle(event);
       }
     }
+
+    public void disableSpeculation() {
+      disabled = true;
+    }
+
   }
 
   private static void validateInputParam(String value, String param)

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

@@ -151,7 +151,7 @@ public class MRClientService extends AbstractService
             + ":" + server.getPort());
     LOG.info("Instantiated MRClientService at " + this.bindAddress);
     try {
-      webApp = WebApps.$for("mapreduce", AppContext.class, appContext).with(conf).
+      webApp = WebApps.$for("mapreduce", AppContext.class, appContext, "ws").with(conf).
           start(new AMWebApp());
     } catch (Exception e) {
       LOG.error("Webapps failed to start. Ignoring for now:", e);

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

@@ -54,6 +54,7 @@ import org.apache.hadoop.mapreduce.jobhistory.JobSubmittedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.JobUnsuccessfulCompletionEvent;
 import org.apache.hadoop.mapreduce.lib.chain.ChainMapper;
 import org.apache.hadoop.mapreduce.lib.chain.ChainReducer;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.security.TokenCache;
 import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager;
@@ -583,13 +584,13 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       if (getState() == JobState.NEW) {
         return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
             appSubmitTime, startTime, finishTime, setupProgress, 0.0f, 0.0f,
-            cleanupProgress, remoteJobConfFile.toString(), amInfos);
+            cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
       }
 
       return MRBuilderUtils.newJobReport(jobId, jobName, username, state,
           appSubmitTime, startTime, finishTime, setupProgress,
           computeProgress(mapTasks), computeProgress(reduceTasks),
-          cleanupProgress, remoteJobConfFile.toString(), amInfos);
+          cleanupProgress, remoteJobConfFile.toString(), amInfos, isUber);
     } finally {
       readLock.unlock();
     }
@@ -812,6 +813,129 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
     return amInfos;
   }
 
+  /**
+   * Decide whether job can be run in uber mode based on various criteria.
+   * @param dataInputLength Total length for all splits
+   */
+  private void makeUberDecision(long dataInputLength) {
+    //FIXME:  need new memory criterion for uber-decision (oops, too late here;
+    // until AM-resizing supported,
+    // must depend on job client to pass fat-slot needs)
+    // these are no longer "system" settings, necessarily; user may override
+    int sysMaxMaps = conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
+
+    //FIXME: handling multiple reduces within a single AM does not seem to
+    //work.
+    // int sysMaxReduces =
+    //     job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
+    int sysMaxReduces = 1;
+
+    long sysMaxBytes = conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
+        conf.getLong("dfs.block.size", 64*1024*1024));  //FIXME: this is
+    // wrong; get FS from [File?]InputFormat and default block size from that
+
+    long sysMemSizeForUberSlot =
+        conf.getInt(MRJobConfig.MR_AM_VMEM_MB,
+            MRJobConfig.DEFAULT_MR_AM_VMEM_MB);
+
+    boolean uberEnabled =
+        conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
+    boolean smallNumMapTasks = (numMapTasks <= sysMaxMaps);
+    boolean smallNumReduceTasks = (numReduceTasks <= sysMaxReduces);
+    boolean smallInput = (dataInputLength <= sysMaxBytes);
+    // ignoring overhead due to UberAM and statics as negligible here:
+    boolean smallMemory =
+        ( (Math.max(conf.getLong(MRJobConfig.MAP_MEMORY_MB, 0),
+            conf.getLong(MRJobConfig.REDUCE_MEMORY_MB, 0))
+            <= sysMemSizeForUberSlot)
+            || (sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT));
+    boolean notChainJob = !isChainJob(conf);
+
+    // User has overall veto power over uberization, or user can modify
+    // limits (overriding system settings and potentially shooting
+    // themselves in the head).  Note that ChainMapper/Reducer are
+    // fundamentally incompatible with MR-1220; they employ a blocking
+    // queue between the maps/reduces and thus require parallel execution,
+    // while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
+    // and thus requires sequential execution.
+    isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
+        && smallInput && smallMemory && notChainJob;
+
+    if (isUber) {
+      LOG.info("Uberizing job " + jobId + ": " + numMapTasks + "m+"
+          + numReduceTasks + "r tasks (" + dataInputLength
+          + " input bytes) will run sequentially on single node.");
+
+      // make sure reduces are scheduled only after all map are completed
+      conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
+                        1.0f);
+      // uber-subtask attempts all get launched on same node; if one fails,
+      // probably should retry elsewhere, i.e., move entire uber-AM:  ergo,
+      // limit attempts to 1 (or at most 2?  probably not...)
+      conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
+      conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
+
+      // disable speculation
+      conf.setBoolean(MRJobConfig.MAP_SPECULATIVE, false);
+      conf.setBoolean(MRJobConfig.REDUCE_SPECULATIVE, false);
+    } else {
+      StringBuilder msg = new StringBuilder();
+      msg.append("Not uberizing ").append(jobId).append(" because:");
+      if (!uberEnabled)
+        msg.append(" not enabled;");
+      if (!smallNumMapTasks)
+        msg.append(" too many maps;");
+      if (!smallNumReduceTasks)
+        msg.append(" too many reduces;");
+      if (!smallInput)
+        msg.append(" too much input;");
+      if (!smallMemory)
+        msg.append(" too much RAM;");
+      if (!notChainJob)
+        msg.append(" chainjob");
+      LOG.info(msg.toString());
+    }
+  }
+
+  /**
+   * ChainMapper and ChainReducer must execute in parallel, so they're not
+   * compatible with uberization/LocalContainerLauncher (100% sequential).
+   */
+  private boolean isChainJob(Configuration conf) {
+    boolean isChainJob = false;
+    try {
+      String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
+      if (mapClassName != null) {
+        Class<?> mapClass = Class.forName(mapClassName);
+        if (ChainMapper.class.isAssignableFrom(mapClass))
+          isChainJob = true;
+      }
+    } catch (ClassNotFoundException cnfe) {
+      // don't care; assume it's not derived from ChainMapper
+    }
+    try {
+      String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
+      if (reduceClassName != null) {
+        Class<?> reduceClass = Class.forName(reduceClassName);
+        if (ChainReducer.class.isAssignableFrom(reduceClass))
+          isChainJob = true;
+      }
+    } catch (ClassNotFoundException cnfe) {
+      // don't care; assume it's not derived from ChainReducer
+    }
+    return isChainJob;
+  }
+
+  /*
+  private int getBlockSize() {
+    String inputClassName = conf.get(MRJobConfig.INPUT_FORMAT_CLASS_ATTR);
+    if (inputClassName != null) {
+      Class<?> inputClass - Class.forName(inputClassName);
+      if (FileInputFormat<K, V>)
+    }
+  }
+  */
+
   public static class InitTransition 
       implements MultipleArcTransition<JobImpl, JobEvent, JobState> {
 
@@ -863,81 +987,8 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
           inputLength += taskSplitMetaInfo[i].getInputDataLength();
         }
 
-        //FIXME:  need new memory criterion for uber-decision (oops, too late here; 
-        // until AM-resizing supported, must depend on job client to pass fat-slot needs)
-        // these are no longer "system" settings, necessarily; user may override
-        int sysMaxMaps = job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXMAPS, 9);
-        int sysMaxReduces =
-            job.conf.getInt(MRJobConfig.JOB_UBERTASK_MAXREDUCES, 1);
-        long sysMaxBytes = job.conf.getLong(MRJobConfig.JOB_UBERTASK_MAXBYTES,
-            job.conf.getLong("dfs.block.size", 64*1024*1024));  //FIXME: this is 
-        // wrong; get FS from [File?]InputFormat and default block size from that
-        //long sysMemSizeForUberSlot = JobTracker.getMemSizeForReduceSlot(); 
-        // FIXME [could use default AM-container memory size...]
-
-        boolean uberEnabled =
-            job.conf.getBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
-        boolean smallNumMapTasks = (job.numMapTasks <= sysMaxMaps);
-        boolean smallNumReduceTasks = (job.numReduceTasks <= sysMaxReduces);
-        boolean smallInput = (inputLength <= sysMaxBytes);
-        boolean smallMemory = true;  //FIXME (see above)
-            // ignoring overhead due to UberTask and statics as negligible here:
-        //  FIXME   && (Math.max(memoryPerMap, memoryPerReduce) <= sysMemSizeForUberSlot
-        //              || sysMemSizeForUberSlot == JobConf.DISABLED_MEMORY_LIMIT)
-        boolean notChainJob = !isChainJob(job.conf);
-
-        // User has overall veto power over uberization, or user can modify
-        // limits (overriding system settings and potentially shooting
-        // themselves in the head).  Note that ChainMapper/Reducer are
-        // fundamentally incompatible with MR-1220; they employ a blocking
-
-        // User has overall veto power over uberization, or user can modify
-        // limits (overriding system settings and potentially shooting
-        // themselves in the head).  Note that ChainMapper/Reducer are
-        // fundamentally incompatible with MR-1220; they employ a blocking
-        // queue between the maps/reduces and thus require parallel execution,
-        // while "uber-AM" (MR AM + LocalContainerLauncher) loops over tasks
-        // and thus requires sequential execution.
-        job.isUber = uberEnabled && smallNumMapTasks && smallNumReduceTasks
-            && smallInput && smallMemory && notChainJob;
-
-        if (job.isUber) {
-          LOG.info("Uberizing job " + job.jobId + ": " + job.numMapTasks + "m+"
-              + job.numReduceTasks + "r tasks (" + inputLength
-              + " input bytes) will run sequentially on single node.");
-              //TODO: also note which node?
-
-          // make sure reduces are scheduled only after all map are completed
-          job.conf.setFloat(MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
-                            1.0f);
-          // uber-subtask attempts all get launched on same node; if one fails,
-          // probably should retry elsewhere, i.e., move entire uber-AM:  ergo,
-          // limit attempts to 1 (or at most 2?  probably not...)
-          job.conf.setInt(MRJobConfig.MAP_MAX_ATTEMPTS, 1);
-          job.conf.setInt(MRJobConfig.REDUCE_MAX_ATTEMPTS, 1);
-
-          // disable speculation:  makes no sense to speculate an entire job
-          //canSpeculateMaps = canSpeculateReduces = false; // [TODO: in old 
-          //version, ultimately was from conf.getMapSpeculativeExecution(), 
-          //conf.getReduceSpeculativeExecution()]
-        } else {
-          StringBuilder msg = new StringBuilder();
-          msg.append("Not uberizing ").append(job.jobId).append(" because:");
-          if (!uberEnabled)
-            msg.append(" not enabled;");
-          if (!smallNumMapTasks)
-            msg.append(" too many maps;");
-          if (!smallNumReduceTasks)
-            msg.append(" too many reduces;");
-          if (!smallInput)
-            msg.append(" too much input;");
-          if (!smallMemory)
-            msg.append(" too much RAM;");
-          if (!notChainJob)
-            msg.append(" chainjob");
-          LOG.info(msg.toString());
-        }
-
+        job.makeUberDecision(inputLength);
+        
         job.taskAttemptCompletionEvents =
             new ArrayList<TaskAttemptCompletionEvent>(
                 job.numMapTasks + job.numReduceTasks + 10);
@@ -1008,35 +1059,6 @@ public class JobImpl implements org.apache.hadoop.mapreduce.v2.app.job.Job,
       }
     }
 
-    /**
-     * ChainMapper and ChainReducer must execute in parallel, so they're not
-     * compatible with uberization/LocalContainerLauncher (100% sequential).
-     */
-    boolean isChainJob(Configuration conf) {
-      boolean isChainJob = false;
-      try {
-        String mapClassName = conf.get(MRJobConfig.MAP_CLASS_ATTR);
-        if (mapClassName != null) {
-          Class<?> mapClass = Class.forName(mapClassName);
-          if (ChainMapper.class.isAssignableFrom(mapClass))
-            isChainJob = true;
-        }
-      } catch (ClassNotFoundException cnfe) {
-        // don't care; assume it's not derived from ChainMapper
-      }
-      try {
-        String reduceClassName = conf.get(MRJobConfig.REDUCE_CLASS_ATTR);
-        if (reduceClassName != null) {
-          Class<?> reduceClass = Class.forName(reduceClassName);
-          if (ChainReducer.class.isAssignableFrom(reduceClass))
-            isChainJob = true;
-        }
-      } catch (ClassNotFoundException cnfe) {
-        // don't care; assume it's not derived from ChainReducer
-      }
-      return isChainJob;
-    }
-
     private void createMapTasks(JobImpl job, long inputLength,
                                 TaskSplitMetaInfo[] splits) {
       for (int i=0; i < job.numMapTasks; ++i) {

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

@@ -60,8 +60,8 @@ public class LocalContainerAllocator extends RMCommunicator
   private static final Log LOG =
       LogFactory.getLog(LocalContainerAllocator.class);
 
+  @SuppressWarnings("rawtypes")
   private final EventHandler eventHandler;
-//  private final ApplicationId appID;
   private AtomicInteger containerCount = new AtomicInteger();
   private long retryInterval;
   private long retrystartTime;
@@ -73,8 +73,6 @@ public class LocalContainerAllocator extends RMCommunicator
                                  AppContext context) {
     super(clientService, context);
     this.eventHandler = context.getEventHandler();
-//    this.appID = context.getApplicationID();
-    
   }
 
   @Override
@@ -88,6 +86,7 @@ public class LocalContainerAllocator extends RMCommunicator
     retrystartTime = System.currentTimeMillis();
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   protected synchronized void heartbeat() throws Exception {
     AllocateRequest allocateRequest = BuilderUtils.newAllocateRequest(
@@ -124,6 +123,7 @@ public class LocalContainerAllocator extends RMCommunicator
     }
   }
 
+  @SuppressWarnings("unchecked")
   @Override
   public void handle(ContainerAllocatorEvent event) {
     if (event.getType() == ContainerAllocator.EventType.CONTAINER_REQ) {

+ 8 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
 import java.io.IOException;
 import java.security.PrivilegedAction;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -65,7 +66,7 @@ public abstract class RMCommunicator extends AbstractService  {
   private int rmPollInterval;//millis
   protected ApplicationId applicationId;
   protected ApplicationAttemptId applicationAttemptId;
-  private volatile boolean stopped;
+  private AtomicBoolean stopped;
   protected Thread allocatorThread;
   protected EventHandler eventHandler;
   protected AMRMProtocol scheduler;
@@ -88,6 +89,7 @@ public abstract class RMCommunicator extends AbstractService  {
     this.eventHandler = context.getEventHandler();
     this.applicationId = context.getApplicationID();
     this.applicationAttemptId = context.getApplicationAttemptId();
+    this.stopped = new AtomicBoolean(false);
   }
 
   @Override
@@ -213,7 +215,10 @@ public abstract class RMCommunicator extends AbstractService  {
 
   @Override
   public void stop() {
-    stopped = true;
+    if (stopped.getAndSet(true)) {
+      // return if already stopped
+      return;
+    }
     allocatorThread.interrupt();
     try {
       allocatorThread.join();
@@ -228,7 +233,7 @@ public abstract class RMCommunicator extends AbstractService  {
     allocatorThread = new Thread(new Runnable() {
       @Override
       public void run() {
-        while (!stopped && !Thread.currentThread().isInterrupted()) {
+        while (!stopped.get() && !Thread.currentThread().isInterrupted()) {
           try {
             Thread.sleep(rmPollInterval);
             try {

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

@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import static org.apache.hadoop.yarn.util.StringHelper.*;
+import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
 
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 
 /**
@@ -29,6 +30,9 @@ public class AMWebApp extends WebApp implements AMParams {
 
   @Override
   public void setup() {
+    bind(JAXBContextResolver.class);
+    bind(GenericExceptionHandler.class);
+    bind(AMWebServices.class);
     route("/", AppController.class);
     route("/app", AppController.class);
     route(pajoin("/job", JOB_ID), AppController.class, "job");

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

@@ -0,0 +1,362 @@
+/**
+ * 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 java.io.IOException;
+
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+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.dao.AppInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+
+import com.google.inject.Inject;
+
+@Path("/ws/v1/mapreduce")
+public class AMWebServices {
+  private final AppContext appCtx;
+  private final App app;
+  private final Configuration conf;
+
+  @Inject
+  public AMWebServices(final App app, final AppContext context,
+      final Configuration conf) {
+    this.appCtx = context;
+    this.app = app;
+    this.conf = conf;
+  }
+
+  Boolean hasAccess(Job job, HttpServletRequest request) {
+    UserGroupInformation callerUgi = UserGroupInformation
+        .createRemoteUser(request.getRemoteUser());
+    if (!job.checkAccess(callerUgi, JobACL.VIEW_JOB)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * check for job access.
+   *
+   * @param job
+   *          the job that is being accessed
+   */
+  void checkAccess(Job job, HttpServletRequest request) {
+    if (!hasAccess(job, request)) {
+      throw new WebApplicationException(Status.UNAUTHORIZED);
+    }
+  }
+
+  @GET
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public AppInfo get() {
+    return getAppInfo();
+  }
+
+  @GET
+  @Path("/info")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public AppInfo getAppInfo() {
+    return new AppInfo(this.app, this.app.context);
+  }
+
+  @GET
+  @Path("/jobs")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobsInfo getJobs(@Context HttpServletRequest hsr) {
+    JobsInfo allJobs = new JobsInfo();
+    for (Job job : appCtx.getAllJobs().values()) {
+      // getAllJobs only gives you a partial we want a full
+      Job fullJob = appCtx.getJob(job.getID());
+      if (fullJob == null) {
+        continue;
+      }
+      allJobs.add(new JobInfo(fullJob, hasAccess(fullJob, hsr)));
+    }
+    return allJobs;
+  }
+
+  @GET
+  @Path("/jobs/{jobid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobInfo getJob(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    return new JobInfo(job, hasAccess(job, hsr));
+
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobCounterInfo getJobCounters(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    return new JobCounterInfo(this.appCtx, job);
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks/{taskid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobTaskCounterInfo getSingleTaskCounters(
+      @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
+      @PathParam("taskid") String tid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = this.appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    return new JobTaskCounterInfo(task);
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/conf")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public ConfInfo getJobConf(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    ConfInfo info;
+    try {
+      info = new ConfInfo(job, this.conf);
+    } catch (IOException e) {
+      throw new NotFoundException("unable to load configuration for job: " + jid);
+    }
+    return info;
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TasksInfo getJobTasks(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid, @QueryParam("type") String type) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TasksInfo allTasks = new TasksInfo();
+    for (Task task : job.getTasks().values()) {
+      TaskType ttype = null;
+      if (type != null && !type.isEmpty()) {
+        try {
+          ttype = MRApps.taskType(type);
+        } catch (YarnException e) {
+          throw new BadRequestException("tasktype must be either m or r");        }
+      }
+      if (ttype != null && task.getType() != ttype) {
+        continue;
+      }
+      allTasks.add(new TaskInfo(task));
+    }
+    return allTasks;
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks/{taskid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskInfo getJobTask(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    return new TaskInfo(task);
+
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks/{taskid}/attempts")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskAttemptsInfo getJobTaskAttempts(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
+    TaskAttemptsInfo attempts = new TaskAttemptsInfo();
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    for (TaskAttempt ta : task.getAttempts().values()) {
+      if (ta != null) {
+        if (task.getType() == TaskType.REDUCE) {
+          attempts.add(new ReduceTaskAttemptInfo(ta, task.getType()));
+        } else {
+          attempts.add(new TaskAttemptInfo(ta, task.getType(), true));
+        }
+      }
+    }
+    return attempts;
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskAttemptInfo getJobTaskAttemptId(@Context HttpServletRequest hsr,
+      @PathParam("jobid") String jid, @PathParam("taskid") String tid,
+      @PathParam("attemptid") String attId) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
+    if (attemptId == null) {
+      throw new NotFoundException("task attempt id " + attId
+          + " not found or invalid");
+    }
+    TaskAttempt ta = task.getAttempt(attemptId);
+    if (ta == null) {
+      throw new NotFoundException("Error getting info on task attempt id "
+          + attId);
+    }
+    if (task.getType() == TaskType.REDUCE) {
+      return new ReduceTaskAttemptInfo(ta, task.getType());
+    } else {
+      return new TaskAttemptInfo(ta, task.getType(), true);
+    }
+  }
+
+  @GET
+  @Path("/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
+      @Context HttpServletRequest hsr, @PathParam("jobid") String jid,
+      @PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = this.appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    checkAccess(job, hsr);
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
+    if (attemptId == null) {
+      throw new NotFoundException("task attempt id " + attId
+          + " not found or invalid");
+    }
+    TaskAttempt ta = task.getAttempt(attemptId);
+    if (ta == null) {
+      throw new NotFoundException("Error getting info on task attempt id "
+          + attId);
+    }
+    return new JobTaskAttemptCounterInfo(ta);
+  }
+}

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

@@ -32,6 +32,7 @@ import org.apache.hadoop.mapreduce.JobACL;
 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.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -72,13 +73,14 @@ public class AppController extends Controller implements AMParams {
    * Render the /info page with an overview of current application.
    */
   public void info() {
+    AppInfo info = new AppInfo(app, app.context);
     info("Application Master Overview").
-      _("Application ID:", $(APP_ID)).
-      _("Application Name:", app.context.getApplicationName()).
-      _("User:", app.context.getUser()).
-      _("Started on:", Times.format(app.context.getStartTime())).
+      _("Application ID:", info.getId()).
+      _("Application Name:", info.getName()).
+      _("User:", info.getUser()).
+      _("Started on:", Times.format(info.getStartTime())).
       _("Elasped: ", org.apache.hadoop.util.StringUtils.formatTime(
-        Times.elapsed(app.context.getStartTime(), 0)));
+          info.getElapsedTime() ));
     render(InfoPage.class);
   }
 

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

@@ -22,14 +22,14 @@ 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.app.webapp.dao.ConfEntryInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
 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;
@@ -71,11 +71,8 @@ public class ConfBlock extends HtmlBlock {
     }
     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));
-    
+      ConfInfo info = new ConfInfo(job, this.conf);
+
       html.div().h3(confPath.toString())._();
       TBODY<TABLE<Hamlet>> tbody = html.
         // Tasks table
@@ -87,10 +84,10 @@ public class ConfBlock extends HtmlBlock {
           _().
         _().
       tbody();
-      for(Map.Entry<String, String> entry : jobConf) {
+      for (ConfEntryInfo entry : info.getProperties()) {
         tbody.
           tr().
-            td(entry.getKey()).
+            td(entry.getName()).
             td(entry.getValue()).
           _();
       }

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

@@ -0,0 +1,77 @@
+/**
+* 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 java.util.Set;
+import java.util.HashSet;
+import java.util.Arrays;
+
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+import com.google.inject.Singleton;
+
+import javax.ws.rs.ext.ContextResolver;
+import javax.ws.rs.ext.Provider;
+import javax.xml.bind.JAXBContext;
+
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.AppInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterGroupInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterGroupInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
+
+@Singleton
+@Provider
+public class JAXBContextResolver implements ContextResolver<JAXBContext> {
+
+  private JAXBContext context;
+  private final Set<Class> types;
+    
+  // you have to specify all the dao classes here
+  private final Class[] cTypes = {AppInfo.class, CounterInfo.class,
+      JobTaskAttemptCounterInfo.class, JobTaskCounterInfo.class,
+      TaskCounterGroupInfo.class, ConfInfo.class, JobCounterInfo.class,
+      TaskCounterInfo.class, CounterGroupInfo.class, JobInfo.class, 
+      JobsInfo.class, ReduceTaskAttemptInfo.class, TaskAttemptInfo.class,
+      TaskInfo.class, TasksInfo.class, TaskAttemptsInfo.class,
+      ConfEntryInfo.class};
+    
+  public JAXBContextResolver() throws Exception {
+    this.types = new HashSet<Class>(Arrays.asList(cTypes));
+    this.context = new JSONJAXBContext(JSONConfiguration.natural().
+        rootUnwrapping(false).build(), cTypes);
+  }
+    
+  @Override
+  public JAXBContext getContext(Class<?> objectType) {
+    return (types.contains(objectType)) ? context : null;
+  }
+}

+ 50 - 144
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java

@@ -18,47 +18,32 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import com.google.inject.Inject;
+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._EVEN;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._INFO_WRAP;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._ODD;
+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._TH;
+
 import java.util.Date;
-import java.util.Map;
 
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
-import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 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.Task;
-import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
-import static org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp.*;
-import static org.apache.hadoop.yarn.util.StringHelper.*;
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+import com.google.inject.Inject;
 
 public class JobBlock extends HtmlBlock {
   final AppContext appContext;
 
-  int runningMapTasks = 0;
-  int pendingMapTasks = 0;
-  int runningReduceTasks = 0;
-  int pendingReduceTasks = 0;
-
-  int newMapAttempts = 0;
-  int runningMapAttempts = 0;
-  int killedMapAttempts = 0;
-  int failedMapAttempts = 0;
-  int successfulMapAttempts = 0;
-  int newReduceAttempts = 0;
-  int runningReduceAttempts = 0;
-  int killedReduceAttempts = 0;
-  int failedReduceAttempts = 0;
-  int successfulReduceAttempts = 0;
-
   @Inject JobBlock(AppContext appctx) {
     appContext = appctx;
   }
@@ -77,23 +62,13 @@ public class JobBlock extends HtmlBlock {
         p()._("Sorry, ", jid, " not found.")._();
       return;
     }
-    JobReport jobReport = job.getReport();
-    String mapPct = percent(jobReport.getMapProgress());
-    String reducePct = percent(jobReport.getReduceProgress());
-    int mapTasks = job.getTotalMaps();
-    int mapTasksComplete = job.getCompletedMaps();
-    int reduceTasks = job.getTotalReduces();
-    int reducesTasksComplete = job.getCompletedReduces();
-    long startTime = jobReport.getStartTime();
-    long finishTime = jobReport.getFinishTime();
-    countTasksAndAttempts(job);
+    JobInfo jinfo = new JobInfo(job, true);
     info("Job Overview").
-        _("Job Name:", job.getName()).
-        _("State:", job.getState()).
-        _("Uberized:", job.isUber()).
-        _("Started:", new Date(startTime)).
-        _("Elapsed:", StringUtils.formatTime(
-            Times.elapsed(startTime, finishTime)));
+        _("Job Name:", jinfo.getName()).
+        _("State:", jinfo.getState()).
+        _("Uberized:", jinfo.isUberized()).
+        _("Started:", new Date(jinfo.getStartTime())).
+        _("Elapsed:", StringUtils.formatTime(jinfo.getElapsedTime()));
     html.
       _(InfoBlock.class).
       div(_INFO_WRAP).
@@ -112,25 +87,25 @@ public class JobBlock extends HtmlBlock {
               a(url("tasks", jid, "m"), "Map")._().
             td().
               div(_PROGRESSBAR).
-                $title(join(mapPct, '%')). // tooltip
+                $title(join(jinfo.getMapProgressPercent(), '%')). // tooltip
                 div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", mapPct, '%'))._()._()._().
-            td(String.valueOf(mapTasks)).
-            td(String.valueOf(pendingMapTasks)).
-            td(String.valueOf(runningMapTasks)).
-            td(String.valueOf(mapTasksComplete))._().
+                  $style(join("width:", jinfo.getMapProgressPercent(), '%'))._()._()._().
+            td(String.valueOf(jinfo.getMapsTotal())).
+            td(String.valueOf(jinfo.getMapsPending())).
+            td(String.valueOf(jinfo.getMapsRunning())).
+            td(String.valueOf(jinfo.getMapsCompleted()))._().
           tr(_EVEN).
             th().
               a(url("tasks", jid, "r"), "Reduce")._().
             td().
               div(_PROGRESSBAR).
-                $title(join(reducePct, '%')). // tooltip
+                $title(join(jinfo.getReduceProgressPercent(), '%')). // tooltip
                 div(_PROGRESSBAR_VALUE).
-                  $style(join("width:", reducePct, '%'))._()._()._().
-            td(String.valueOf(reduceTasks)).
-            td(String.valueOf(pendingReduceTasks)).
-            td(String.valueOf(runningReduceTasks)).
-            td(String.valueOf(reducesTasksComplete))._()
+                  $style(join("width:", jinfo.getReduceProgressPercent(), '%'))._()._()._().
+            td(String.valueOf(jinfo.getReducesTotal())).
+            td(String.valueOf(jinfo.getReducesPending())).
+            td(String.valueOf(jinfo.getReducesRunning())).
+            td(String.valueOf(jinfo.getReducesCompleted()))._()
           ._().
 
         // Attempts table
@@ -145,110 +120,41 @@ public class JobBlock extends HtmlBlock {
         tr(_ODD).
           th("Maps").
           td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.NEW.toString()), 
-              String.valueOf(newMapAttempts))._().
+              TaskAttemptStateUI.NEW.toString()),
+              String.valueOf(jinfo.getNewMapAttempts()))._().
           td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.RUNNING.toString()), 
-              String.valueOf(runningMapAttempts))._().
+              TaskAttemptStateUI.RUNNING.toString()),
+              String.valueOf(jinfo.getRunningMapAttempts()))._().
           td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(failedMapAttempts))._().
+              TaskAttemptStateUI.FAILED.toString()),
+              String.valueOf(jinfo.getFailedMapAttempts()))._().
           td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(killedMapAttempts))._().
+              TaskAttemptStateUI.KILLED.toString()),
+              String.valueOf(jinfo.getKilledMapAttempts()))._().
           td().a(url("attempts", jid, "m",
-              TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(successfulMapAttempts))._().
+              TaskAttemptStateUI.SUCCESSFUL.toString()),
+              String.valueOf(jinfo.getSuccessfulMapAttempts()))._().
         _().
         tr(_EVEN).
           th("Reduces").
           td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.NEW.toString()), 
-              String.valueOf(newReduceAttempts))._().
+              TaskAttemptStateUI.NEW.toString()),
+              String.valueOf(jinfo.getNewReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.RUNNING.toString()), 
-              String.valueOf(runningReduceAttempts))._().
+              TaskAttemptStateUI.RUNNING.toString()),
+              String.valueOf(jinfo.getRunningReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(failedReduceAttempts))._().
+              TaskAttemptStateUI.FAILED.toString()),
+              String.valueOf(jinfo.getFailedReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(killedReduceAttempts))._().
+              TaskAttemptStateUI.KILLED.toString()),
+              String.valueOf(jinfo.getKilledReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
-              TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(successfulReduceAttempts))._().
+              TaskAttemptStateUI.SUCCESSFUL.toString()),
+              String.valueOf(jinfo.getSuccessfulReduceAttempts()))._().
          _().
        _().
      _();
   }
 
-  private void countTasksAndAttempts(Job job) {
-    Map<TaskId, Task> tasks = job.getTasks();
-    for (Task task : tasks.values()) {
-      switch (task.getType()) {
-      case MAP:
-        // Task counts
-        switch (task.getState()) {
-        case RUNNING:
-          ++runningMapTasks;
-          break;
-        case SCHEDULED:
-          ++pendingMapTasks;
-          break;
-        }
-        break;
-      case REDUCE:
-        // Task counts
-        switch (task.getState()) {
-        case RUNNING:
-          ++runningReduceTasks;
-          break;
-        case SCHEDULED:
-          ++pendingReduceTasks;
-          break;
-        }
-        break;
-      }
-
-      // Attempts counts
-      Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
-      for (TaskAttempt attempt : attempts.values()) {
-
-        int newAttempts = 0, running = 0, successful = 0, failed = 0, killed =0;
-
-        if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
-          ++newAttempts;
-        } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
-            .getState())) {
-          ++running;
-        } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
-            .getState())) {
-          ++successful;
-        } else if (TaskAttemptStateUI.FAILED
-            .correspondsTo(attempt.getState())) {
-          ++failed;
-        } else if (TaskAttemptStateUI.KILLED
-            .correspondsTo(attempt.getState())) {
-          ++killed;
-        }
-
-        switch (task.getType()) {
-        case MAP:
-          newMapAttempts += newAttempts;
-          runningMapAttempts += running;
-          successfulMapAttempts += successful;
-          failedMapAttempts += failed;
-          killedMapAttempts += killed;
-          break;
-        case REDUCE:
-          newReduceAttempts += newAttempts;
-          runningReduceAttempts += running;
-          successfulReduceAttempts += successful;
-          failedReduceAttempts += failed;
-          killedReduceAttempts += killed;
-          break;
-        }
-      }
-    }
-  }
 }

+ 23 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobsBlock.java

@@ -18,18 +18,19 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import com.google.inject.Inject;
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
 
-import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 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.mapreduce.v2.app.webapp.dao.JobInfo;
 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.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
-import static org.apache.hadoop.yarn.util.StringHelper.*;
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+import com.google.inject.Inject;
 
 public class JobsBlock extends HtmlBlock {
   final AppContext appContext;
@@ -54,38 +55,31 @@ public class JobsBlock extends HtmlBlock {
             th("Reduces Total").
             th("Reduces Completed")._()._().
         tbody();
-    for (Job job : appContext.getAllJobs().values()) {
-      String jobID = MRApps.toString(job.getID());
-      JobReport report = job.getReport();
-      String mapPct = percent(report.getMapProgress());
-      String mapsTotal = String.valueOf(job.getTotalMaps());
-      String mapsCompleted = String.valueOf(job.getCompletedMaps());
-      String reducePct = percent(report.getReduceProgress());
-      String reduceTotal = String.valueOf(job.getTotalReduces());
-      String reduceCompleted = String.valueOf(job.getCompletedReduces());
+    for (Job j : appContext.getAllJobs().values()) {
+      JobInfo job = new JobInfo(j, false);
       tbody.
         tr().
           td().
-            span().$title(String.valueOf(job.getID().getId()))._(). // for sorting
-            a(url("job", jobID), jobID)._().
-          td(job.getName().toString()).
-          td(job.getState().toString()).
+            span().$title(String.valueOf(job.getId()))._(). // for sorting
+            a(url("job", job.getId()), job.getId())._().
+          td(job.getName()).
+          td(job.getState()).
           td().
-            span().$title(mapPct)._(). // for sorting
+            span().$title(job.getMapProgressPercent())._(). // for sorting
             div(_PROGRESSBAR).
-              $title(join(mapPct, '%')). // tooltip
+              $title(join(job.getMapProgressPercent(), '%')). // tooltip
               div(_PROGRESSBAR_VALUE).
-                $style(join("width:", mapPct, '%'))._()._()._().
-          td(mapsTotal).
-          td(mapsCompleted).
+                $style(join("width:", job.getMapProgressPercent(), '%'))._()._()._().
+          td(String.valueOf(job.getMapsTotal())).
+          td(String.valueOf(job.getMapsCompleted())).
           td().
-            span().$title(reducePct)._(). // for sorting
+            span().$title(job.getReduceProgressPercent())._(). // for sorting
             div(_PROGRESSBAR).
-              $title(join(reducePct, '%')). // tooltip
+              $title(join(job.getReduceProgressPercent(), '%')). // tooltip
               div(_PROGRESSBAR_VALUE).
-                $style(join("width:", reducePct, '%'))._()._()._().
-          td(reduceTotal).
-          td(reduceCompleted)._();
+                $style(join("width:", job.getReduceProgressPercent(), '%'))._()._()._().
+          td(String.valueOf(job.getReducesTotal())).
+          td(String.valueOf(job.getReducesCompleted()))._();
     }
     tbody._()._();
   }

+ 26 - 18
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TaskPage.java

@@ -18,23 +18,29 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import java.util.Collection;
+import static org.apache.hadoop.yarn.util.StringHelper.percent;
+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.tableInit;
 
-import com.google.common.base.Joiner;
-import com.google.inject.Inject;
+import java.util.Collection;
 
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
-import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.Times;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.*;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TD;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TR;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
-import static org.apache.hadoop.yarn.util.StringHelper.*;
-import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
+
+import com.google.inject.Inject;
 
 public class TaskPage extends AppView {
 
@@ -66,24 +72,26 @@ public class TaskPage extends AppView {
             th(".tsh", "Elapsed").
             th(".note", "Note")._()._().
         tbody();
-      for (TaskAttempt ta : getTaskAttempts()) {
-        String taid = MRApps.toString(ta.getID());
-        String progress = percent(ta.getProgress());
-        ContainerId containerId = ta.getAssignedContainerID();
+      for (TaskAttempt attempt : getTaskAttempts()) {
+        TaskAttemptInfo ta = new TaskAttemptInfo(attempt, true);
+        String taid = ta.getId();
+        String progress = percent(ta.getProgress() / 100);
+        ContainerId containerId = ta.getAssignedContainerId();
 
-        String nodeHttpAddr = ta.getNodeHttpAddress();
-        long startTime = ta.getLaunchTime();
+        String nodeHttpAddr = ta.getNode();
+        long startTime = ta.getStartTime();
         long finishTime = ta.getFinishTime();
-        long elapsed = Times.elapsed(startTime, finishTime);
+        long elapsed = ta.getElapsedTime();
+        String diag = ta.getNote() == null ? "" : ta.getNote();
         TD<TR<TBODY<TABLE<Hamlet>>>> nodeTd = tbody.
           tr().
             td(".id", taid).
             td(".progress", progress).
-            td(".state", ta.getState().toString()).
+            td(".state", ta.getState()).
             td().
               a(".nodelink", url("http://", nodeHttpAddr), nodeHttpAddr);
         if (containerId != null) {
-          String containerIdStr = ConverterUtils.toString(containerId);
+          String containerIdStr = ta.getAssignedContainerIdStr();
           nodeTd._(" ").
             a(".logslink", url("http://", nodeHttpAddr, "node", "containerlogs",
               containerIdStr, app.getJob().getUserName()), "logs");
@@ -92,7 +100,7 @@ public class TaskPage extends AppView {
           td(".ts", Times.format(startTime)).
           td(".ts", Times.format(finishTime)).
           td(".dt", StringUtils.formatTime(elapsed)).
-          td(".note", Joiner.on('\n').join(ta.getDiagnostics()))._();
+          td(".note", diag)._();
       }
       tbody._()._();
     }

+ 17 - 14
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/TasksBlock.java

@@ -18,21 +18,24 @@
 
 package org.apache.hadoop.mapreduce.v2.app.webapp;
 
-import com.google.inject.Inject;
+import static org.apache.hadoop.mapreduce.v2.app.webapp.AMParams.TASK_TYPE;
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.util.StringHelper.percent;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI._PROGRESSBAR_VALUE;
 
-import org.apache.hadoop.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.webapp.dao.TaskInfo;
 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.*;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
-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 com.google.inject.Inject;
 
 public class TasksBlock extends HtmlBlock {
   final App app;
@@ -67,16 +70,16 @@ public class TasksBlock extends HtmlBlock {
       if (type != null && task.getType() != type) {
         continue;
       }
-      String tid = MRApps.toString(task.getID());
-      TaskReport report = task.getReport();
-      String pct = percent(report.getProgress());
-      long startTime = report.getStartTime();
-      long finishTime = report.getFinishTime();
-      long elapsed = Times.elapsed(startTime, finishTime);
+      TaskInfo info = new TaskInfo(task);
+      String tid = info.getId();
+      String pct = percent(info.getProgress() / 100);
+      long startTime = info.getStartTime();
+      long finishTime = info.getFinishTime();
+      long elapsed = info.getElapsedTime();
       tbody.
         tr().
           td().
-            br().$title(String.valueOf(task.getID().getId()))._(). // sorting
+            br().$title(String.valueOf(info.getTaskNum()))._(). // sorting
             a(url("task", tid), tid)._().
           td().
             br().$title(pct)._().
@@ -84,7 +87,7 @@ public class TasksBlock extends HtmlBlock {
               $title(join(pct, '%')). // tooltip
               div(_PROGRESSBAR_VALUE).
                 $style(join("width:", pct, '%'))._()._()._().
-          td(report.getTaskState().toString()).
+          td(info.getState()).
           td().
             br().$title(String.valueOf(startTime))._().
             _(Times.format(startTime))._().

+ 70 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/AppInfo.java

@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.v2.app.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.webapp.App;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "info")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AppInfo {
+
+  protected String appId;
+  protected String name;
+  protected String user;
+  protected String hostname;
+  protected long startedOn;
+  protected long elapsedTime;
+
+  public AppInfo() {
+  }
+
+  public AppInfo(App app, AppContext context) {
+    this.appId = context.getApplicationID().toString();
+    this.name = context.getApplicationName().toString();
+    this.user = context.getUser().toString();
+    this.startedOn = context.getStartTime();
+    this.elapsedTime = Times.elapsed(context.getStartTime(), 0);
+  }
+
+  public String getId() {
+    return this.appId;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public String getUser() {
+    return this.user;
+  }
+
+  public long getStartTime() {
+    return this.startedOn;
+  }
+
+  public long getElapsedTime() {
+    return this.elapsedTime;
+  }
+
+}

+ 46 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfEntryInfo.java

@@ -0,0 +1,46 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ConfEntryInfo {
+
+  protected String name;
+  protected String value;
+
+  public ConfEntryInfo() {
+  }
+
+  public ConfEntryInfo(String key, String value) {
+    this.name = key;
+    this.value = value;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public String getValue() {
+    return this.value;
+  }
+}

+ 66 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ConfInfo.java

@@ -0,0 +1,66 @@
+/**
+ * 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.dao;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ConfInfo {
+
+  protected String path;
+  protected ArrayList<ConfEntryInfo> property;
+
+  public ConfInfo() {
+  }
+
+  public ConfInfo(Job job, Configuration conf) throws IOException {
+
+    Path confPath = job.getConfFile();
+    this.property = new ArrayList<ConfEntryInfo>();
+    // 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));
+    this.path = confPath.toString();
+    for (Map.Entry<String, String> entry : jobConf) {
+      this.property.add(new ConfEntryInfo(entry.getKey(), entry.getValue()));
+    }
+
+  }
+
+  public ArrayList<ConfEntryInfo> getProperties() {
+    return this.property;
+  }
+
+  public String getPath() {
+    return this.path;
+  }
+
+}

+ 54 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterGroupInfo.java

@@ -0,0 +1,54 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.api.records.Counter;
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+
+@XmlRootElement(name = "counterGroup")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CounterGroupInfo {
+
+  protected String counterGroupName;
+  @XmlElement(name = "counter")
+  protected ArrayList<CounterInfo> counter;
+
+  public CounterGroupInfo() {
+  }
+
+  public CounterGroupInfo(String name, CounterGroup g, CounterGroup mg,
+      CounterGroup rg) {
+    this.counterGroupName = name;
+    this.counter = new ArrayList<CounterInfo>();
+
+    for (Counter c : g.getAllCounters().values()) {
+      Counter mc = mg == null ? null : mg.getCounter(c.getName());
+      Counter rc = rg == null ? null : rg.getCounter(c.getName());
+      CounterInfo cinfo = new CounterInfo(c, mc, rc);
+      this.counter.add(cinfo);
+    }
+  }
+
+}

+ 44 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/CounterInfo.java

@@ -0,0 +1,44 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.api.records.Counter;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class CounterInfo {
+
+  protected String counterName;
+  protected long totalCounterValue;
+  protected long mapCounterValue;
+  protected long reduceCounterValue;
+
+  public CounterInfo() {
+  }
+
+  public CounterInfo(Counter counter, Counter mc, Counter rc) {
+    this.counterName = counter.getName();
+    this.totalCounterValue = counter.getValue();
+    this.mapCounterValue = mc == null ? 0 : mc.getValue();
+    this.reduceCounterValue = rc == null ? 0 : rc.getValue();
+  }
+}

+ 100 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobCounterInfo.java

@@ -0,0 +1,100 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+import org.apache.hadoop.mapreduce.v2.api.records.Counters;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+import org.apache.hadoop.mapreduce.v2.app.job.Job;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.app.job.impl.JobImpl;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+
+@XmlRootElement(name = "jobCounters")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobCounterInfo {
+
+  @XmlTransient
+  protected Counters total = null;
+  @XmlTransient
+  protected Counters map = null;
+  @XmlTransient
+  protected Counters reduce = null;
+
+  protected String id;
+  protected ArrayList<CounterGroupInfo> counterGroups;
+
+  public JobCounterInfo() {
+  }
+
+  public JobCounterInfo(AppContext ctx, Job job) {
+    getCounters(ctx, job);
+    counterGroups = new ArrayList<CounterGroupInfo>();
+    this.id = MRApps.toString(job.getID());
+
+    int numGroups = 0;
+
+    if (total != null) {
+      for (CounterGroup g : total.getAllCounterGroups().values()) {
+        if (g != null) {
+          CounterGroup mg = map == null ? null : map.getCounterGroup(g
+              .getName());
+          CounterGroup rg = reduce == null ? null : reduce.getCounterGroup(g
+              .getName());
+          ++numGroups;
+
+          CounterGroupInfo cginfo = new CounterGroupInfo(g.getName(), g, mg, rg);
+          counterGroups.add(cginfo);
+        }
+      }
+    }
+  }
+
+  private void getCounters(AppContext ctx, Job job) {
+    total = JobImpl.newCounters();
+    if (job == null) {
+      return;
+    }
+    map = JobImpl.newCounters();
+    reduce = JobImpl.newCounters();
+    // Get all types of counters
+    Map<TaskId, Task> tasks = job.getTasks();
+    for (Task t : tasks.values()) {
+      Counters counters = t.getCounters();
+      JobImpl.incrAllCounters(total, counters);
+      switch (t.getType()) {
+      case MAP:
+        JobImpl.incrAllCounters(map, counters);
+        break;
+      case REDUCE:
+        JobImpl.incrAllCounters(reduce, counters);
+        break;
+      }
+    }
+  }
+
+}

+ 349 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobInfo.java

@@ -0,0 +1,349 @@
+/**
+ * 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.dao;
+
+import static org.apache.hadoop.yarn.util.StringHelper.percent;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.JobACL;
+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.TaskId;
+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.util.MRApps;
+import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.security.authorize.AccessControlList;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "job")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobInfo {
+
+  // ok for any user to see
+  protected long startTime;
+  protected long finishTime;
+  protected long elapsedTime;
+  protected String id;
+  protected String name;
+  protected String user;
+  protected String state;
+  protected int mapsTotal;
+  protected int mapsCompleted;
+  protected float mapProgress;
+  protected int reducesTotal;
+  protected int reducesCompleted;
+  protected float reduceProgress;
+
+  @XmlTransient
+  protected String mapProgressPercent;
+  @XmlTransient
+  protected String reduceProgressPercent;
+
+  // these should only be seen if acls allow
+  protected int mapsPending;
+  protected int mapsRunning;
+  protected int reducesPending;
+  protected int reducesRunning;
+  protected boolean uberized;
+  protected String diagnostics;
+  protected int newReduceAttempts = 0;
+  protected int runningReduceAttempts = 0;
+  protected int failedReduceAttempts = 0;
+  protected int killedReduceAttempts = 0;
+  protected int successfulReduceAttempts = 0;
+  protected int newMapAttempts = 0;
+  protected int runningMapAttempts = 0;
+  protected int failedMapAttempts = 0;
+  protected int killedMapAttempts = 0;
+  protected int successfulMapAttempts = 0;
+  protected ArrayList<ConfEntryInfo> acls;
+
+  @XmlTransient
+  protected int numMaps;
+  @XmlTransient
+  protected int numReduces;
+
+  public JobInfo() {
+  }
+
+  public JobInfo(Job job, Boolean hasAccess) {
+    this.id = MRApps.toString(job.getID());
+    JobReport report = job.getReport();
+    countTasksAndAttempts(job);
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    this.elapsedTime = Times.elapsed(this.startTime, this.finishTime);
+    if (this.elapsedTime == -1) {
+      this.elapsedTime = 0;
+    }
+    this.name = job.getName().toString();
+    this.user = job.getUserName();
+    this.state = job.getState().toString();
+    this.mapsTotal = job.getTotalMaps();
+    this.mapsCompleted = job.getCompletedMaps();
+    this.mapProgress = report.getMapProgress() * 100;
+    this.mapProgressPercent = percent(report.getMapProgress());
+    this.reducesTotal = job.getTotalReduces();
+    this.reducesCompleted = job.getCompletedReduces();
+    this.reduceProgress = report.getReduceProgress() * 100;
+    this.reduceProgressPercent = percent(report.getReduceProgress());
+
+    this.acls = new ArrayList<ConfEntryInfo>();
+    if (hasAccess) {
+      this.uberized = job.isUber();
+
+      List<String> diagnostics = job.getDiagnostics();
+      if (diagnostics != null && !diagnostics.isEmpty()) {
+        StringBuffer b = new StringBuffer();
+        for (String diag : diagnostics) {
+          b.append(diag);
+        }
+        this.diagnostics = b.toString();
+      }
+
+      Map<JobACL, AccessControlList> allacls = job.getJobACLs();
+      if (allacls != null) {
+        for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
+          this.acls.add(new ConfEntryInfo(entry.getKey().getAclName(), entry
+              .getValue().getAclString()));
+        }
+      }
+    }
+  }
+
+  public int getNewReduceAttempts() {
+    return this.newReduceAttempts;
+  }
+
+  public int getKilledReduceAttempts() {
+    return this.killedReduceAttempts;
+  }
+
+  public int getFailedReduceAttempts() {
+    return this.failedReduceAttempts;
+  }
+
+  public int getRunningReduceAttempts() {
+    return this.runningReduceAttempts;
+  }
+
+  public int getSuccessfulReduceAttempts() {
+    return this.successfulReduceAttempts;
+  }
+
+  public int getNewMapAttempts() {
+    return this.newMapAttempts;
+  }
+
+  public int getKilledMapAttempts() {
+    return this.killedMapAttempts;
+  }
+
+  public ArrayList<ConfEntryInfo> getAcls() {
+    return acls;
+  }
+
+  public int getFailedMapAttempts() {
+    return this.failedMapAttempts;
+  }
+
+  public int getRunningMapAttempts() {
+    return this.runningMapAttempts;
+  }
+
+  public int getSuccessfulMapAttempts() {
+    return this.successfulMapAttempts;
+  }
+
+  public int getReducesCompleted() {
+    return this.reducesCompleted;
+  }
+
+  public int getReducesTotal() {
+    return this.reducesTotal;
+  }
+
+  public int getReducesPending() {
+    return this.reducesPending;
+  }
+
+  public int getReducesRunning() {
+    return this.reducesRunning;
+  }
+
+  public int getMapsCompleted() {
+    return this.mapsCompleted;
+  }
+
+  public int getMapsTotal() {
+    return this.mapsTotal;
+  }
+
+  public int getMapsPending() {
+    return this.mapsPending;
+  }
+
+  public int getMapsRunning() {
+    return this.mapsRunning;
+  }
+
+  public String getState() {
+    return this.state;
+  }
+
+  public String getUser() {
+    return this.user;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public String getId() {
+    return this.id;
+  }
+
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public long getElapsedTime() {
+    return this.elapsedTime;
+  }
+
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public boolean isUberized() {
+    return this.uberized;
+  }
+
+  public String getdiagnostics() {
+    return this.diagnostics;
+  }
+
+  public float getMapProgress() {
+    return this.mapProgress;
+  }
+
+  public String getMapProgressPercent() {
+    return this.mapProgressPercent;
+  }
+
+  public float getReduceProgress() {
+    return this.reduceProgress;
+  }
+
+  public String getReduceProgressPercent() {
+    return this.reduceProgressPercent;
+  }
+
+  /**
+   * Go through a job and update the member variables with counts for
+   * information to output in the page.
+   * 
+   * @param job
+   *          the job to get counts for.
+   */
+  private void countTasksAndAttempts(Job job) {
+    numReduces = 0;
+    numMaps = 0;
+    final Map<TaskId, Task> tasks = job.getTasks();
+    if (tasks == null) {
+      return;
+    }
+    for (Task task : tasks.values()) {
+      switch (task.getType()) {
+      case MAP:
+        // Task counts
+        switch (task.getState()) {
+        case RUNNING:
+          ++this.mapsRunning;
+          break;
+        case SCHEDULED:
+          ++this.mapsPending;
+          break;
+        }
+        break;
+      case REDUCE:
+        // Task counts
+        switch (task.getState()) {
+        case RUNNING:
+          ++this.reducesRunning;
+          break;
+        case SCHEDULED:
+          ++this.reducesPending;
+          break;
+        }
+        break;
+      }
+      // Attempts counts
+      Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
+      int newAttempts, running, successful, failed, killed;
+      for (TaskAttempt attempt : attempts.values()) {
+
+        newAttempts = 0;
+        running = 0;
+        successful = 0;
+        failed = 0;
+        killed = 0;
+        if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
+          ++newAttempts;
+        } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt.getState())) {
+          ++running;
+        } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
+            .getState())) {
+          ++successful;
+        } else if (TaskAttemptStateUI.FAILED.correspondsTo(attempt.getState())) {
+          ++failed;
+        } else if (TaskAttemptStateUI.KILLED.correspondsTo(attempt.getState())) {
+          ++killed;
+        }
+
+        switch (task.getType()) {
+        case MAP:
+          this.newMapAttempts += newAttempts;
+          this.runningMapAttempts += running;
+          this.successfulMapAttempts += successful;
+          this.failedMapAttempts += failed;
+          this.killedMapAttempts += killed;
+          break;
+        case REDUCE:
+          this.newReduceAttempts += newAttempts;
+          this.runningReduceAttempts += running;
+          this.successfulReduceAttempts += successful;
+          this.failedReduceAttempts += failed;
+          this.killedReduceAttempts += killed;
+          break;
+        }
+      }
+    }
+  }
+
+}

+ 63 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskAttemptCounterInfo.java

@@ -0,0 +1,63 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+import org.apache.hadoop.mapreduce.v2.api.records.Counters;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+
+@XmlRootElement(name = "JobTaskAttemptCounters")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobTaskAttemptCounterInfo {
+
+  @XmlTransient
+  protected Counters total = null;
+
+  protected String id;
+  protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
+
+  public JobTaskAttemptCounterInfo() {
+  }
+
+  public JobTaskAttemptCounterInfo(TaskAttempt taskattempt) {
+
+    long value = 0;
+    this.id = MRApps.toString(taskattempt.getID());
+    total = taskattempt.getCounters();
+    taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
+    if (total != null) {
+      for (CounterGroup g : total.getAllCounterGroups().values()) {
+        if (g != null) {
+          TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
+          if (cginfo != null) {
+            taskCounterGroups.add(cginfo);
+          }
+        }
+      }
+    }
+  }
+}

+ 59 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobTaskCounterInfo.java

@@ -0,0 +1,59 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+import org.apache.hadoop.mapreduce.v2.api.records.Counters;
+import org.apache.hadoop.mapreduce.v2.app.job.Task;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+
+@XmlRootElement(name = "jobTaskCounters")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobTaskCounterInfo {
+
+  @XmlTransient
+  protected Counters total = null;
+
+  protected String id;
+  protected ArrayList<TaskCounterGroupInfo> taskCounterGroups;
+
+  public JobTaskCounterInfo() {
+  }
+
+  public JobTaskCounterInfo(Task task) {
+    total = task.getCounters();
+    this.id = MRApps.toString(task.getID());
+    taskCounterGroups = new ArrayList<TaskCounterGroupInfo>();
+    if (total != null) {
+      for (CounterGroup g : total.getAllCounterGroups().values()) {
+        if (g != null) {
+          TaskCounterGroupInfo cginfo = new TaskCounterGroupInfo(g.getName(), g);
+          taskCounterGroups.add(cginfo);
+        }
+      }
+    }
+  }
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/JobsInfo.java

@@ -0,0 +1,43 @@
+/**
+ * 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 joblicable 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "jobs")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobsInfo {
+
+  protected ArrayList<JobInfo> job = new ArrayList<JobInfo>();
+
+  public JobsInfo() {
+  } // JAXB needs this
+
+  public void add(JobInfo jobInfo) {
+    job.add(jobInfo);
+  }
+
+  public ArrayList<JobInfo> getJobs() {
+    return job;
+  }
+
+}

+ 83 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/ReduceTaskAttemptInfo.java

@@ -0,0 +1,83 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "taskAttempt")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ReduceTaskAttemptInfo extends TaskAttemptInfo {
+
+  protected long shuffleFinishTime;
+  protected long mergeFinishTime;
+  protected long elapsedShuffleTime;
+  protected long elapsedMergeTime;
+  protected long elapsedReduceTime;
+
+  public ReduceTaskAttemptInfo() {
+  }
+
+  public ReduceTaskAttemptInfo(TaskAttempt ta, TaskType type) {
+    super(ta, type, false);
+
+    this.shuffleFinishTime = ta.getShuffleFinishTime();
+    this.mergeFinishTime = ta.getSortFinishTime();
+    this.elapsedShuffleTime = Times.elapsed(this.startTime,
+        this.shuffleFinishTime, false);
+    if (this.elapsedShuffleTime == -1) {
+      this.elapsedShuffleTime = 0;
+    }
+    this.elapsedMergeTime = Times.elapsed(this.shuffleFinishTime,
+        this.mergeFinishTime, false);
+    if (this.elapsedMergeTime == -1) {
+      this.elapsedMergeTime = 0;
+    }
+    this.elapsedReduceTime = Times.elapsed(this.mergeFinishTime,
+        this.finishTime, false);
+    if (this.elapsedReduceTime == -1) {
+      this.elapsedReduceTime = 0;
+    }
+  }
+
+  public long getShuffleFinishTime() {
+    return this.shuffleFinishTime;
+  }
+
+  public long getMergeFinishTime() {
+    return this.mergeFinishTime;
+  }
+
+  public long getElapsedShuffleTime() {
+    return this.elapsedShuffleTime;
+  }
+
+  public long getElapsedMergeTime() {
+    return this.elapsedMergeTime;
+  }
+
+  public long getElapsedReduceTime() {
+    return this.elapsedReduceTime;
+  }
+}

+ 133 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptInfo.java

@@ -0,0 +1,133 @@
+/**
+ * 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.dao;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlSeeAlso;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "taskAttempt")
+@XmlSeeAlso({ ReduceTaskAttemptInfo.class })
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TaskAttemptInfo {
+
+  protected long startTime;
+  protected long finishTime;
+  protected long elapsedTime;
+  protected float progress;
+  protected String id;
+  protected String rack;
+  protected String state;
+  protected String nodeHttpAddress;
+  protected String diagnostics;
+  protected String type;
+  protected String assignedContainerId;
+
+  @XmlTransient
+  protected ContainerId assignedContainer;
+
+  public TaskAttemptInfo() {
+  }
+
+  public TaskAttemptInfo(TaskAttempt ta, Boolean isRunning) {
+    this(ta, TaskType.MAP, isRunning);
+  }
+
+  public TaskAttemptInfo(TaskAttempt ta, TaskType type, Boolean isRunning) {
+    this.type = type.toString();
+    this.id = MRApps.toString(ta.getID());
+    this.nodeHttpAddress = ta.getNodeHttpAddress();
+    this.startTime = ta.getLaunchTime();
+    this.finishTime = ta.getFinishTime();
+    this.assignedContainerId = ConverterUtils.toString(ta
+        .getAssignedContainerID());
+    this.assignedContainer = ta.getAssignedContainerID();
+    this.progress = ta.getProgress() * 100;
+    this.state = ta.getState().toString();
+    this.elapsedTime = Times
+        .elapsed(this.startTime, this.finishTime, isRunning);
+    if (this.elapsedTime == -1) {
+      this.elapsedTime = 0;
+    }
+    List<String> diagnostics = ta.getDiagnostics();
+    if (diagnostics != null && !diagnostics.isEmpty()) {
+      StringBuffer b = new StringBuffer();
+      for (String diag : diagnostics) {
+        b.append(diag);
+      }
+      this.diagnostics = b.toString();
+    }
+    this.rack = ta.getNodeRackName();
+  }
+
+  public String getAssignedContainerIdStr() {
+    return this.assignedContainerId;
+  }
+
+  public ContainerId getAssignedContainerId() {
+    return this.assignedContainer;
+  }
+
+  public String getState() {
+    return this.state;
+  }
+
+  public String getId() {
+    return this.id;
+  }
+
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public float getProgress() {
+    return this.progress;
+  }
+
+  public long getElapsedTime() {
+    return this.elapsedTime;
+  }
+
+  public String getNode() {
+    return this.nodeHttpAddress;
+  }
+
+  public String getRack() {
+    return this.rack;
+  }
+
+  public String getNote() {
+    return this.diagnostics;
+  }
+
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskAttemptsInfo.java

@@ -0,0 +1,43 @@
+/**
+ * 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 taskattemptlicable 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "taskattempts")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TaskAttemptsInfo {
+
+  protected ArrayList<TaskAttemptInfo> taskattempt = new ArrayList<TaskAttemptInfo>();
+
+  public TaskAttemptsInfo() {
+  } // JAXB needs this
+
+  public void add(TaskAttemptInfo taskattemptInfo) {
+    taskattempt.add(taskattemptInfo);
+  }
+
+  public ArrayList<TaskAttemptInfo> getTaskAttempts() {
+    return taskattempt;
+  }
+
+}

+ 49 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskCounterGroupInfo.java

@@ -0,0 +1,49 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.mapreduce.v2.api.records.Counter;
+import org.apache.hadoop.mapreduce.v2.api.records.CounterGroup;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TaskCounterGroupInfo {
+
+  protected String counterGroupName;
+  protected ArrayList<TaskCounterInfo> counter;
+
+  public TaskCounterGroupInfo() {
+  }
+
+  public TaskCounterGroupInfo(String name, CounterGroup g) {
+    this.counterGroupName = name;
+    this.counter = new ArrayList<TaskCounterInfo>();
+
+    for (Counter c : g.getAllCounters().values()) {
+      TaskCounterInfo cinfo = new TaskCounterInfo(c.getName(), c.getValue());
+      this.counter.add(cinfo);
+    }
+  }
+}

+ 46 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskCounterInfo.java

@@ -0,0 +1,46 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "counter")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TaskCounterInfo {
+
+  protected String name;
+  protected long value;
+
+  public TaskCounterInfo() {
+  }
+
+  public TaskCounterInfo(String name, long value) {
+    this.name = name;
+    this.value = value;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public long getValue() {
+    return value;
+  }
+}

+ 122 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TaskInfo.java

@@ -0,0 +1,122 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+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.util.MRApps;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "task")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TaskInfo {
+
+  protected long startTime;
+  protected long finishTime;
+  protected long elapsedTime;
+  protected float progress;
+  protected String id;
+  protected String state;
+  protected String type;
+  protected String successfulAttempt;
+
+  @XmlTransient
+  int taskNum;
+
+  @XmlTransient
+  TaskAttempt successful;
+
+  public TaskInfo() {
+  }
+
+  public TaskInfo(Task task) {
+    TaskType ttype = task.getType();
+    this.type = ttype.toString();
+    TaskReport report = task.getReport();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    this.elapsedTime = Times.elapsed(this.startTime, this.finishTime, false);
+    if (this.elapsedTime == -1) {
+      this.elapsedTime = 0;
+    }
+    this.state = report.getTaskState().toString();
+    this.progress = report.getProgress() * 100;
+    this.id = MRApps.toString(task.getID());
+    this.taskNum = task.getID().getId();
+    this.successful = getSuccessfulAttempt(task);
+    if (successful != null) {
+      this.successfulAttempt = MRApps.toString(successful.getID());
+    } else {
+      this.successfulAttempt = "";
+    }
+  }
+
+  public float getProgress() {
+    return this.progress;
+  }
+
+  public String getState() {
+    return this.state;
+  }
+
+  public String getId() {
+    return this.id;
+  }
+
+  public int getTaskNum() {
+    return this.taskNum;
+  }
+
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public long getElapsedTime() {
+    return this.elapsedTime;
+  }
+
+  public String getSuccessfulAttempt() {
+    return this.successfulAttempt;
+  }
+
+  public TaskAttempt getSuccessful() {
+    return this.successful;
+  }
+
+  private TaskAttempt getSuccessfulAttempt(Task task) {
+    for (TaskAttempt attempt : task.getAttempts().values()) {
+      if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
+        return attempt;
+      }
+    }
+    return null;
+  }
+
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/dao/TasksInfo.java

@@ -0,0 +1,43 @@
+/**
+ * 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 tasklicable 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "tasks")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class TasksInfo {
+
+  protected ArrayList<TaskInfo> task = new ArrayList<TaskInfo>();
+
+  public TasksInfo() {
+  } // JAXB needs this
+
+  public void add(TaskInfo taskInfo) {
+    task.add(taskInfo);
+  }
+
+  public ArrayList<TaskInfo> getTasks() {
+    return task;
+  }
+
+}

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

@@ -183,6 +183,7 @@ public class TestMRClientService {
     Assert.assertEquals(1, amInfo.getContainerId().getApplicationAttemptId()
         .getAttemptId());
     Assert.assertTrue(amInfo.getStartTime() > 0);
+    Assert.assertEquals(false, jr.isUber());
   }
   
   private void verifyTaskAttemptReport(TaskAttemptReport tar) {

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

@@ -118,7 +118,7 @@ public class TestRMContainerAllocator {
     Job mockJob = mock(Job.class);
     when(mockJob.getReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0, 
-            0, 0, 0, 0, 0, 0, "jobfile", null));
+            0, 0, 0, 0, 0, 0, "jobfile", null, false));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob);
 
@@ -195,7 +195,7 @@ public class TestRMContainerAllocator {
     Job mockJob = mock(Job.class);
     when(mockJob.getReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
-            0, 0, 0, 0, 0, 0, "jobfile", null));
+            0, 0, 0, 0, 0, 0, "jobfile", null, false));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob);
 
@@ -261,7 +261,7 @@ public class TestRMContainerAllocator {
     Job mockJob = mock(Job.class);
     when(mockJob.getReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
-            0, 0, 0, 0, 0, 0, "jobfile", null));
+            0, 0, 0, 0, 0, 0, "jobfile", null, false));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob);
 
@@ -375,7 +375,7 @@ public class TestRMContainerAllocator {
     public JobReport getReport() {
       return MRBuilderUtils.newJobReport(this.jobId, "job", "user",
           JobState.RUNNING, 0, 0, 0, this.setupProgress, this.mapProgress,
-          this.reduceProgress, this.cleanupProgress, "jobfile", null);
+          this.reduceProgress, this.cleanupProgress, "jobfile", null, false);
     }
   }
 
@@ -511,7 +511,7 @@ public class TestRMContainerAllocator {
     Job mockJob = mock(Job.class);
     when(mockJob.getReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
-            0, 0, 0, 0, 0, 0, "jobfile", null));
+            0, 0, 0, 0, 0, 0, "jobfile", null, false));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob);
 
@@ -610,7 +610,7 @@ public class TestRMContainerAllocator {
     Job mockJob = mock(Job.class);
     when(mockJob.getReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "job", "user", JobState.RUNNING, 0,
-            0, 0, 0, 0, 0, 0, "jobfile", null));
+            0, 0, 0, 0, 0, 0, "jobfile", null, false));
     MyContainerAllocator allocator = new MyContainerAllocator(rm, conf,
         appAttemptId, mockJob);
 

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

@@ -754,8 +754,6 @@ public class TestRuntimeEstimators {
   }
 
   class MyAppContext implements AppContext {
-    // I'll be making Avro objects by hand.  Please don't do that very often.
-
     private final ApplicationAttemptId myAppAttemptID;
     private final ApplicationId myApplicationID;
     private final JobId myJobID;

+ 0 - 153
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/avro/MRClientProtocol.genavro

@@ -1,153 +0,0 @@
-@namespace("org.apache.hadoop.mapreduce.v2.api")
-protocol MRClientProtocol {
-
-  import idl "./yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  enum TaskType {
-    MAP,
-    REDUCE
-  }
-
-  record JobID {
-    org.apache.hadoop.yarn.ApplicationID appID;
-    int id;
-  }
-
-  record TaskID {
-    JobID jobID;
-    TaskType taskType;
-    int id;
-  }
-
-  record TaskAttemptID {
-    TaskID taskID; 
-    int id;
-  }
-
-  enum TaskState {
-    NEW,
-    SCHEDULED,
-    RUNNING,
-    SUCCEEDED,
-    FAILED,
-    KILL_WAIT,
-    KILLED
-  }
-
-  enum Phase {
-    STARTING,
-    MAP,
-    SHUFFLE,
-    SORT,
-    REDUCE,
-    CLEANUP
-  }
-
-  record Counter {
-    string name;
-    string displayName;
-    long value; 
-  }
-
-  record CounterGroup {
-    string name;
-    string displayname;
-    map<Counter> counters;
-  }
-
-  record Counters {
-    map<CounterGroup> groups;
-  }
-
-  record TaskReport {
-    TaskID id;
-    TaskState state;
-    float progress;
-    long startTime;
-    long finishTime;
-    Counters counters;
-    array<TaskAttemptID> runningAttempts;
-    union{TaskAttemptID, null} successfulAttempt;
-    array<string> diagnostics;
-  }
-
-  enum TaskAttemptState {
-    NEW,
-    UNASSIGNED,
-    ASSIGNED,
-    RUNNING,
-    COMMIT_PENDING,
-    SUCCESS_CONTAINER_CLEANUP,
-    SUCCEEDED,
-    FAIL_CONTAINER_CLEANUP,
-    FAIL_TASK_CLEANUP,
-    FAILED,
-    KILL_CONTAINER_CLEANUP,
-    KILL_TASK_CLEANUP,
-    KILLED
-  }
-
-  record TaskAttemptReport {
-    TaskAttemptID id;
-    TaskAttemptState state;
-    float progress;
-    long startTime;
-    long finishTime;
-    Counters counters;
-    string diagnosticInfo;
-    string stateString;
-    Phase phase;
-  }
-
-  enum JobState {
-    NEW,
-    INITED,
-    RUNNING,
-    SUCCEEDED,
-    FAILED,
-    KILL_WAIT,
-    KILLED,
-    ERROR
-  }
-
-  record JobReport {
-    JobID id;
-    JobState state;
-    float mapProgress;
-    float reduceProgress;
-    float cleanupProgress;
-    float setupProgress;
-    long startTime;
-    long finishTime;
-  }
-
-  enum TaskAttemptCompletionEventStatus {
-    FAILED,
-    KILLED,
-    SUCCEEDED,
-    OBSOLETE,
-    TIPFAILED
-  }
-
-  record TaskAttemptCompletionEvent {
-    TaskAttemptID attemptId;
-    TaskAttemptCompletionEventStatus status;
-    string mapOutputServerAddress;
-    int attemptRunTime;
-    int eventId;
-  }
-
-  JobReport getJobReport(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  TaskReport getTaskReport(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  TaskAttemptReport getTaskAttemptReport(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  Counters getCounters(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<TaskAttemptCompletionEvent> getTaskAttemptCompletionEvents(JobID jobID, int fromEventId, int maxEvents) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<TaskReport> getTaskReports(JobID jobID, TaskType taskType) throws org.apache.hadoop.yarn.YarnRemoteException;
-  array<string> getDiagnostics(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-
-  void killJob(JobID jobID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void killTask(TaskID taskID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void killTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-  void failTaskAttempt(TaskAttemptID taskAttemptID) throws org.apache.hadoop.yarn.YarnRemoteException;
-
-}

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java

@@ -288,7 +288,7 @@ public class TypeConverter {
             .getMapProgress(), jobreport.getReduceProgress(), jobreport
             .getCleanupProgress(), fromYarn(jobreport.getJobState()),
         jobPriority, jobreport.getUser(), jobreport.getJobName(), jobreport
-            .getJobFile(), trackingUrl);
+            .getJobFile(), trackingUrl, jobreport.isUber());
     jobStatus.setFailureInfo(jobreport.getDiagnostics());
     return jobStatus;
   }
@@ -421,7 +421,7 @@ public class TypeConverter {
           TypeConverter.fromYarn(application.getYarnApplicationState()),
           org.apache.hadoop.mapreduce.JobPriority.NORMAL,
           application.getUser(), application.getName(),
-          application.getQueue(), jobFile, trackingUrl
+          application.getQueue(), jobFile, trackingUrl, false
       );
     jobStatus.setSchedulingInfo(trackingUrl); // Set AM tracking url
     jobStatus.setStartTime(application.getStartTime());

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

@@ -36,6 +36,7 @@ public interface JobReport {
   public abstract String getDiagnostics();
   public abstract String getJobFile();
   public abstract List<AMInfo> getAMInfos();
+  public abstract boolean isUber();
 
   public abstract void setJobId(JobId jobId);
   public abstract void setJobState(JobState jobState);
@@ -52,4 +53,5 @@ public interface JobReport {
   public abstract void setDiagnostics(String diagnostics);
   public abstract void setJobFile(String jobFile);
   public abstract void setAMInfos(List<AMInfo> amInfos);
+  public abstract void setIsUber(boolean isUber);
 }

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

@@ -332,4 +332,16 @@ public class JobReportPBImpl extends ProtoBase<JobReportProto> implements
   private JobState convertFromProtoFormat(JobStateProto e) {
     return MRProtoUtils.convertFromProtoFormat(e);
   }
+
+  @Override
+  public synchronized boolean isUber() {
+    JobReportProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getIsUber();
+  }
+
+  @Override
+  public synchronized void setIsUber(boolean isUber) {
+    maybeInitBuilder();
+    builder.setIsUber(isUber);
+  }
 }  

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

@@ -60,7 +60,8 @@ public class MRBuilderUtils {
   public static JobReport newJobReport(JobId jobId, String jobName,
       String userName, JobState state, long submitTime, long startTime, long finishTime,
       float setupProgress, float mapProgress, float reduceProgress,
-      float cleanupProgress, String jobFile, List<AMInfo> amInfos) {
+      float cleanupProgress, String jobFile, List<AMInfo> amInfos,
+      boolean isUber) {
     JobReport report = Records.newRecord(JobReport.class);
     report.setJobId(jobId);
     report.setJobName(jobName);
@@ -75,6 +76,7 @@ public class MRBuilderUtils {
     report.setReduceProgress(reduceProgress);
     report.setJobFile(jobFile);
     report.setAMInfos(amInfos);
+    report.setIsUber(isUber);
     return report;
   }
 

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

@@ -152,6 +152,7 @@ message JobReportProto {
   optional string jobFile = 13;
   repeated AMInfoProto am_infos = 14;
   optional int64 submit_time = 15;
+  optional bool is_uber = 16 [default = false];
 }
 
 message AMInfoProto {

+ 9 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobQueueClient.java

@@ -109,7 +109,14 @@ class JobQueueClient extends Configured implements Tool {
     return exitcode;
   }
 
+// format and print information about the passed in job queue.
+  void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer)
+    throws IOException {
+    printJobQueueInfo(jobQueueInfo, writer, "");
+  }
+
   // format and print information about the passed in job queue.
+  @SuppressWarnings("deprecation")
   void printJobQueueInfo(JobQueueInfo jobQueueInfo, Writer writer,
     String prefix) throws IOException {
     if (jobQueueInfo == null) {
@@ -136,7 +143,7 @@ class JobQueueClient extends Configured implements Tool {
   private void displayQueueList() throws IOException {
     JobQueueInfo[] rootQueues = jc.getRootQueues();
     for (JobQueueInfo queue : rootQueues) {
-      printJobQueueInfo(queue, new PrintWriter(System.out), "");
+      printJobQueueInfo(queue, new PrintWriter(System.out));
     }
   }
   
@@ -174,7 +181,7 @@ class JobQueueClient extends Configured implements Tool {
       System.out.println("Queue \"" + queue + "\" does not exist.");
       return;
     }
-    printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out), "");
+    printJobQueueInfo(jobQueueInfo, new PrintWriter(System.out));
     if (showJobs && (jobQueueInfo.getChildren() == null ||
         jobQueueInfo.getChildren().size() == 0)) {
       JobStatus[] jobs = jc.getJobsFromQueue(queue);

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

@@ -97,7 +97,7 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
                    String user, String jobName, 
                    String jobFile, String trackingUrl) {
     this(jobid, mapProgress, reduceProgress, cleanupProgress, runState,
-                  JobPriority.NORMAL, user, jobName, jobFile, trackingUrl);
+        JobPriority.NORMAL, user, jobName, jobFile, trackingUrl);
   }
 
   /**
@@ -135,7 +135,8 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
                       String user, String jobName, String jobFile, 
                       String trackingUrl) {
      this(jobid, 0.0f, mapProgress, reduceProgress, 
-          cleanupProgress, runState, jp, user, jobName, jobFile, trackingUrl);
+          cleanupProgress, runState, jp, user, jobName, jobFile,
+          trackingUrl);
    }
    
   /**
@@ -157,10 +158,57 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
                     int runState, JobPriority jp, String user, String jobName, 
                     String jobFile, String trackingUrl) {
      this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
-         runState, jp,
-         user, jobName, "default", jobFile, trackingUrl);
+         runState, jp, user, jobName, "default", jobFile, trackingUrl);
    }
+
+   /**
+    * Create a job status object for a given jobid.
+    * @param jobid The jobid of the job
+    * @param setupProgress The progress made on the setup
+    * @param mapProgress The progress made on the maps
+    * @param reduceProgress The progress made on the reduces
+    * @param cleanupProgress The progress made on the cleanup
+    * @param runState The current state of the job
+    * @param jp Priority of the job.
+    * @param user userid of the person who submitted the job.
+    * @param jobName user-specified job name.
+    * @param jobFile job configuration file. 
+    * @param trackingUrl link to the web-ui for details of the job.
+    * @param isUber Whether job running in uber mode
+    */
+    public JobStatus(JobID jobid, float setupProgress, float mapProgress,
+                     float reduceProgress, float cleanupProgress, 
+                     int runState, JobPriority jp, String user, String jobName, 
+                     String jobFile, String trackingUrl, boolean isUber) {
+      this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
+          runState, jp, user, jobName, "default", jobFile, trackingUrl, isUber);
+    }   
    
+   /**
+    * Create a job status object for a given jobid.
+    * @param jobid The jobid of the job
+    * @param setupProgress The progress made on the setup
+    * @param mapProgress The progress made on the maps
+    * @param reduceProgress The progress made on the reduces
+    * @param cleanupProgress The progress made on the cleanup
+    * @param runState The current state of the job
+    * @param jp Priority of the job.
+    * @param user userid of the person who submitted the job.
+    * @param jobName user-specified job name.
+    * @param queue job queue name.
+    * @param jobFile job configuration file.
+    * @param trackingUrl link to the web-ui for details of the job.
+    */
+   public JobStatus(JobID jobid, float setupProgress, float mapProgress,
+       float reduceProgress, float cleanupProgress,
+       int runState, JobPriority jp,
+       String user, String jobName, String queue,
+       String jobFile, String trackingUrl) {
+     this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
+         runState, jp,
+         user, jobName, queue, jobFile, trackingUrl, false);
+   }
+
    /**
     * Create a job status object for a given jobid.
     * @param jobid The jobid of the job
@@ -175,25 +223,25 @@ public class JobStatus extends org.apache.hadoop.mapreduce.JobStatus {
     * @param queue job queue name.
     * @param jobFile job configuration file. 
     * @param trackingUrl link to the web-ui for details of the job.
+    * @param isUber Whether job running in uber mode
     */
    public JobStatus(JobID jobid, float setupProgress, float mapProgress,
        float reduceProgress, float cleanupProgress, 
        int runState, JobPriority jp, 
        String user, String jobName, String queue, 
-       String jobFile, String trackingUrl) {
+       String jobFile, String trackingUrl, boolean isUber) {
      super(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
          getEnum(runState), org.apache.hadoop.mapreduce.JobPriority.valueOf(jp.name()),
-         user, jobName, queue, jobFile, trackingUrl);
+         user, jobName, queue, jobFile, trackingUrl, isUber);
    }
 
-
   public static JobStatus downgrade(org.apache.hadoop.mapreduce.JobStatus stat){
     JobStatus old = new JobStatus(JobID.downgrade(stat.getJobID()),
       stat.getSetupProgress(), stat.getMapProgress(), stat.getReduceProgress(),
       stat.getCleanupProgress(), stat.getState().getValue(), 
       JobPriority.valueOf(stat.getPriority().name()),
       stat.getUsername(), stat.getJobName(), stat.getJobFile(),
-      stat.getTrackingUrl());
+      stat.getTrackingUrl(), stat.isUber());
     old.setStartTime(stat.getStartTime());
     old.setFinishTime(stat.getFinishTime());
     old.setSchedulingInfo(stat.getSchedulingInfo());

+ 15 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java

@@ -467,6 +467,7 @@ public class Job extends JobContextImpl implements JobContext {
     sb.append("Job File: ").append(status.getJobFile()).append("\n");
     sb.append("Job Tracking URL : ").append(status.getTrackingUrl());
     sb.append("\n");
+    sb.append("Uber job : ").append(status.isUber()).append("\n");
     sb.append("map() completion: ");
     sb.append(status.getMapProgress()).append("\n");
     sb.append("reduce() completion: ");
@@ -1268,12 +1269,20 @@ public class Job extends JobContextImpl implements JobContext {
       Job.getProgressPollInterval(clientConf);
     /* make sure to report full progress after the job is done */
     boolean reportedAfterCompletion = false;
+    boolean reportedUberMode = false;
     while (!isComplete() || !reportedAfterCompletion) {
       if (isComplete()) {
         reportedAfterCompletion = true;
       } else {
         Thread.sleep(progMonitorPollIntervalMillis);
       }
+      if (status.getState() == JobStatus.State.PREP) {
+        continue;
+      }      
+      if (!reportedUberMode) {
+        reportedUberMode = true;
+        LOG.info("Job " + jobId + " running in uber mode : " + isUber());
+      }      
       String report = 
         (" map " + StringUtils.formatPercent(mapProgress(), 0)+
             " reduce " + 
@@ -1497,4 +1506,10 @@ public class Job extends JobContextImpl implements JobContext {
     conf.set(Job.OUTPUT_FILTER, newValue.toString());
   }
 
+  public boolean isUber() throws IOException, InterruptedException {
+    ensureState(JobState.RUNNING);
+    updateStatus();
+    return status.isUber();
+  }
+  
 }

+ 55 - 10
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/JobStatus.java

@@ -97,7 +97,7 @@ public class JobStatus implements Writable, Cloneable {
   private int usedMem;
   private int reservedMem;
   private int neededMem;
-
+  private boolean isUber;
     
   /**
    */
@@ -115,17 +115,17 @@ public class JobStatus implements Writable, Cloneable {
    * @param jp Priority of the job.
    * @param user userid of the person who submitted the job.
    * @param jobName user-specified job name.
-   * @param jobFile job configuration file. 
+   * @param jobFile job configuration file.
    * @param trackingUrl link to the web-ui for details of the job.
    */
    public JobStatus(JobID jobid, float setupProgress, float mapProgress,
-                    float reduceProgress, float cleanupProgress, 
+                    float reduceProgress, float cleanupProgress,
                     State runState, JobPriority jp, String user, String jobName, 
                     String jobFile, String trackingUrl) {
      this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress, 
-         runState, jp, user, jobName, "default", jobFile, trackingUrl);
+         runState, jp, user, jobName, "default", jobFile, trackingUrl, false);
    }
-           
+
    /**
     * Create a job status object for a given jobid.
     * @param jobid The jobid of the job
@@ -138,14 +138,39 @@ public class JobStatus implements Writable, Cloneable {
     * @param user userid of the person who submitted the job.
     * @param jobName user-specified job name.
     * @param queue queue name
-    * @param jobFile job configuration file. 
+    * @param jobFile job configuration file.
     * @param trackingUrl link to the web-ui for details of the job.
     */
     public JobStatus(JobID jobid, float setupProgress, float mapProgress,
-                     float reduceProgress, float cleanupProgress, 
-                     State runState, JobPriority jp, 
-                     String user, String jobName, String queue, 
+                     float reduceProgress, float cleanupProgress,
+                     State runState, JobPriority jp,
+                     String user, String jobName, String queue,
                      String jobFile, String trackingUrl) {
+      this(jobid, setupProgress, mapProgress, reduceProgress, cleanupProgress,
+          runState, jp, user, jobName, queue, jobFile, trackingUrl, false);
+    }
+
+   /**
+    * Create a job status object for a given jobid.
+    * @param jobid The jobid of the job
+    * @param setupProgress The progress made on the setup
+    * @param mapProgress The progress made on the maps
+    * @param reduceProgress The progress made on the reduces
+    * @param cleanupProgress The progress made on the cleanup
+    * @param runState The current state of the job
+    * @param jp Priority of the job.
+    * @param user userid of the person who submitted the job.
+    * @param jobName user-specified job name.
+    * @param queue queue name
+    * @param jobFile job configuration file.
+    * @param trackingUrl link to the web-ui for details of the job.
+    * @param isUber Whether job running in uber mode
+    */
+    public JobStatus(JobID jobid, float setupProgress, float mapProgress,
+                     float reduceProgress, float cleanupProgress,
+                     State runState, JobPriority jp,
+                     String user, String jobName, String queue,
+                     String jobFile, String trackingUrl, boolean isUber) {
       this.jobid = jobid;
       this.setupProgress = setupProgress;
       this.mapProgress = mapProgress;
@@ -161,8 +186,9 @@ public class JobStatus implements Writable, Cloneable {
       this.jobName = jobName;
       this.jobFile = jobFile;
       this.trackingUrl = trackingUrl;
+      this.isUber = isUber;
     }
-    
+
 
   /**
    * Sets the map progress of this job
@@ -411,6 +437,7 @@ public class JobStatus implements Writable, Cloneable {
     Text.writeString(out, jobName);
     Text.writeString(out, trackingUrl);
     Text.writeString(out, jobFile);
+    out.writeBoolean(isUber);
 
     // Serialize the job's ACLs
     out.writeInt(jobACLs.size());
@@ -438,6 +465,7 @@ public class JobStatus implements Writable, Cloneable {
     this.jobName = Text.readString(in);
     this.trackingUrl = Text.readString(in);
     this.jobFile = Text.readString(in);
+    this.isUber = in.readBoolean();
 
     // De-serialize the job's ACLs
     int numACLs = in.readInt();
@@ -562,9 +590,26 @@ public class JobStatus implements Writable, Cloneable {
     this.neededMem = n;
   }
 
+  /**
+   * Whether job running in uber mode
+   * @return job in uber-mode
+   */
+  public synchronized boolean isUber() {
+    return isUber;
+  }
+  
+  /**
+   * Set uber-mode flag 
+   * @param isUber Whether job running in uber-mode
+   */
+  public synchronized void setUber(boolean isUber) {
+    this.isUber = isUber;
+  }
+  
   public String toString() {
     StringBuffer buffer = new StringBuffer();
     buffer.append("job-id : " + jobid);
+    buffer.append("uber-mode : " + isUber);
     buffer.append("map-progress : " + mapProgress);
     buffer.append("reduce-progress : " + reduceProgress);
     buffer.append("cleanup-progress : " + cleanupProgress);

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

@@ -150,6 +150,10 @@ public interface MRJobConfig {
 
   public static final String NUM_REDUCE_PROFILES = "mapreduce.task.profile.reduces";
 
+  public static final String TASK_MAP_PROFILE_PARAMS = "mapreduce.task.profile.map.params";
+  
+  public static final String TASK_REDUCE_PROFILE_PARAMS = "mapreduce.task.profile.reduce.params";
+  
   public static final String TASK_TIMEOUT = "mapreduce.task.timeout";
 
   public static final String TASK_ID = "mapreduce.task.id";
@@ -298,12 +302,6 @@ public interface MRJobConfig {
     "mapreduce.job.ubertask.maxreduces";
   public static final String JOB_UBERTASK_MAXBYTES =
     "mapreduce.job.ubertask.maxbytes";
-  public static final String UBERTASK_JAVA_OPTS =
-    "mapreduce.ubertask.child.java.opts";  // or mapreduce.uber.java.opts?
-  public static final String UBERTASK_ULIMIT =
-    "mapreduce.ubertask.child.ulimit";     // or mapreduce.uber.ulimit?
-  public static final String UBERTASK_ENV =
-    "mapreduce.ubertask.child.env";        // or mapreduce.uber.env?
 
   public static final String MR_PREFIX = "yarn.app.mapreduce.";
 

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

@@ -76,6 +76,7 @@ public abstract class AbstractCounters<C extends Counter,
                   TaskCounter.class.getName());
     legacyMap.put("org.apache.hadoop.mapred.JobInProgress$Counter",
                   JobCounter.class.getName());
+    legacyMap.put("FileSystemCounter", FileSystemCounter.class.getName());
   }
 
   private final Limits limits = new Limits();

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

@@ -353,7 +353,7 @@ public class JobHistoryParser {
    * The class where job information is aggregated into after parsing
    */
   public static class JobInfo {
-    String errorInfo = "None";
+    String errorInfo = "";
     long submitTime;
     long finishTime;
     JobID jobid;

+ 11 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestCounters.java

@@ -22,6 +22,8 @@ import static org.junit.Assert.assertEquals;
 import java.io.IOException;
 import java.text.ParseException;
 
+import org.apache.hadoop.mapred.Counters.Counter;
+import org.apache.hadoop.mapreduce.FileSystemCounter;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.TaskCounter;
 import org.junit.Test;
@@ -102,6 +104,7 @@ public class TestCounters {
     Counters counters = new Counters();
     counters.incrCounter(TaskCounter.MAP_INPUT_RECORDS, 1);
     counters.incrCounter(JobCounter.DATA_LOCAL_MAPS, 1);
+    counters.findCounter("file", FileSystemCounter.BYTES_READ).increment(1);
     
     assertEquals("New name", 1, counters.findCounter(
         TaskCounter.class.getName(), "MAP_INPUT_RECORDS").getValue());
@@ -114,6 +117,14 @@ public class TestCounters {
     assertEquals("Legacy name", 1, counters.findCounter(
         "org.apache.hadoop.mapred.JobInProgress$Counter",
         "DATA_LOCAL_MAPS").getValue());
+
+    assertEquals("New name", 1, counters.findCounter(
+        FileSystemCounter.class.getName(), "FILE_BYTES_READ").getValue());
+    assertEquals("New name and method", 1, counters.findCounter("file",
+        FileSystemCounter.BYTES_READ).getValue());
+    assertEquals("Legacy name", 1, counters.findCounter(
+        "FileSystemCounter",
+        "FILE_BYTES_READ").getValue());
   }
   
   public static void main(String[] args) throws IOException {

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

@@ -45,7 +45,7 @@ public class TestJobQueueClient {
 
     ByteArrayOutputStream bbos = new ByteArrayOutputStream();
     PrintWriter writer = new PrintWriter(bbos);
-    queueClient.printJobQueueInfo(parent, writer, "");
+    queueClient.printJobQueueInfo(parent, writer);
 
     Assert.assertTrue("printJobQueueInfo did not print grandchild's name",
       bbos.toString().contains("GrandChildQueue"));

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

@@ -63,17 +63,20 @@ public class TestJobMonitorAndPrint extends TestCase {
     when(cluster.getConf()).thenReturn(conf);
     when(cluster.getClient()).thenReturn(clientProtocol);
     JobStatus jobStatus = new JobStatus(new JobID("job_000", 1), 0f, 0f, 0f, 0f, 
-        State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
+        State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", 
+        "tmp-jobfile", "tmp-url");
     job = Job.getInstance(cluster, jobStatus, conf);
     job = spy(job);
   }
 
   @Test
   public void testJobMonitorAndPrint() throws Exception {
-    JobStatus jobStatus_1 = new JobStatus(new JobID("job_000", 1), 1f, 0.1f, 0.1f, 0f, 
-        State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
-    JobStatus jobStatus_2 = new JobStatus(new JobID("job_000", 1), 1f, 1f, 1f, 1f, 
-        State.SUCCEEDED, JobPriority.HIGH, "tmp-user", "tmp-jobname", "tmp-jobfile", "tmp-url");
+    JobStatus jobStatus_1 = new JobStatus(new JobID("job_000", 1), 1f, 0.1f,
+        0.1f, 0f, State.RUNNING, JobPriority.HIGH, "tmp-user", "tmp-jobname",
+        "tmp-queue", "tmp-jobfile", "tmp-url", true);
+    JobStatus jobStatus_2 = new JobStatus(new JobID("job_000", 1), 1f, 1f,
+        1f, 1f, State.SUCCEEDED, JobPriority.HIGH, "tmp-user", "tmp-jobname",
+        "tmp-queue", "tmp-jobfile", "tmp-url", true);
 
     doAnswer(
         new Answer<TaskCompletionEvent[]>() {
@@ -102,15 +105,21 @@ public class TestJobMonitorAndPrint extends TestCase {
     String line;
     boolean foundHundred = false;
     boolean foundComplete = false;
-    String match_1 = "map 100% reduce 100%";
-    String match_2 = "completed successfully";
+    boolean foundUber = false;
+    String match_1 = "uber mode : true";
+    String match_2 = "map 100% reduce 100%";
+    String match_3 = "completed successfully";
     while ((line = r.readLine()) != null) {
-      foundHundred = line.contains(match_1);
+      if (line.contains(match_1)) {
+        foundUber = true;
+      }
+      foundHundred = line.contains(match_2);      
       if (foundHundred)
         break;
     }
     line = r.readLine();
-    foundComplete = line.contains(match_2);
+    foundComplete = line.contains(match_3);
+    assertTrue(foundUber);
     assertTrue(foundHundred);
     assertTrue(foundComplete);
   }

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

@@ -107,6 +107,7 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
     report.setTrackingUrl(JobHistoryUtils.getHistoryUrl(conf, TypeConverter
         .toYarn(TypeConverter.fromYarn(jobId)).getAppId()));
     report.setAMInfos(getAMInfos());
+    report.setIsUber(isUber());
   }
 
   @Override

+ 19 - 20
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java

@@ -27,12 +27,11 @@ import java.security.PrivilegedExceptionAction;
 import java.util.Arrays;
 import java.util.Collection;
 
-import org.apache.hadoop.ipc.Server;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.mapreduce.JobACL;
 import org.apache.hadoop.mapreduce.v2.api.MRClientProtocol;
 import org.apache.hadoop.mapreduce.v2.api.protocolrecords.FailTaskAttemptRequest;
@@ -79,14 +78,14 @@ import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
 
 /**
- * This module is responsible for talking to the 
+ * This module is responsible for talking to the
  * JobClient (user facing).
  *
  */
 public class HistoryClientService extends AbstractService {
 
   private static final Log LOG = LogFactory.getLog(HistoryClientService.class);
-  
+
   private MRClientProtocol protocolHandler;
   private Server server;
   private WebApp webApp;
@@ -118,22 +117,22 @@ public class HistoryClientService extends AbstractService {
     server =
         rpc.getServer(MRClientProtocol.class, protocolHandler, address,
             conf, null,
-            conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT, 
+            conf.getInt(JHAdminConfig.MR_HISTORY_CLIENT_THREAD_COUNT,
                 JHAdminConfig.DEFAULT_MR_HISTORY_CLIENT_THREAD_COUNT));
-    
+
     // Enable service authorization?
     if (conf.getBoolean(
-        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION, 
+        CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION,
         false)) {
       server.refreshServiceAcl(conf, new MRAMPolicyProvider());
     }
-    
+
     server.start();
     this.bindAddress =
         NetUtils.createSocketAddr(hostNameResolved.getHostAddress()
             + ":" + server.getPort());
     LOG.info("Instantiated MRClientService at " + this.bindAddress);
-    
+
     super.start();
   }
 
@@ -141,7 +140,7 @@ public class HistoryClientService extends AbstractService {
     webApp = new HsWebApp(history);
     String bindAddress = conf.get(JHAdminConfig.MR_HISTORY_WEBAPP_ADDRESS,
         JHAdminConfig.DEFAULT_MR_HISTORY_WEBAPP_ADDRESS);
-    WebApps.$for("jobhistory", this).with(conf).at(bindAddress).start(webApp); 
+    WebApps.$for("jobhistory", HistoryClientService.class, this, "ws").with(conf).at(bindAddress).start(webApp);
   }
 
   @Override
@@ -158,7 +157,7 @@ public class HistoryClientService extends AbstractService {
   private class MRClientProtocolHandler implements MRClientProtocol {
 
     private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-    
+
     private Job verifyAndGetJob(final JobId jobID) throws YarnRemoteException {
       UserGroupInformation loginUgi = null;
       Job job = null;
@@ -194,7 +193,7 @@ public class HistoryClientService extends AbstractService {
       response.setCounters(job.getCounters());
       return response;
     }
-    
+
     @Override
     public GetJobReportResponse getJobReport(GetJobReportRequest request) throws YarnRemoteException {
       JobId jobId = request.getJobId();
@@ -227,23 +226,23 @@ public class HistoryClientService extends AbstractService {
       JobId jobId = request.getJobId();
       int fromEventId = request.getFromEventId();
       int maxEvents = request.getMaxEvents();
-      
+
       Job job = verifyAndGetJob(jobId);
       GetTaskAttemptCompletionEventsResponse response = recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
       response.addAllCompletionEvents(Arrays.asList(job.getTaskAttemptCompletionEvents(fromEventId, maxEvents)));
       return response;
     }
-      
+
     @Override
     public KillJobResponse killJob(KillJobRequest request) throws YarnRemoteException {
       throw RPCUtil.getRemoteException("Invalid operation on completed job");
     }
-    
+
     @Override
     public KillTaskResponse killTask(KillTaskRequest request) throws YarnRemoteException {
       throw RPCUtil.getRemoteException("Invalid operation on completed job");
     }
-    
+
     @Override
     public KillTaskAttemptResponse killTaskAttempt(KillTaskAttemptRequest request) throws YarnRemoteException {
       throw RPCUtil.getRemoteException("Invalid operation on completed job");
@@ -252,15 +251,15 @@ public class HistoryClientService extends AbstractService {
     @Override
     public GetDiagnosticsResponse getDiagnostics(GetDiagnosticsRequest request) throws YarnRemoteException {
       TaskAttemptId taskAttemptId = request.getTaskAttemptId();
-    
+
       Job job = verifyAndGetJob(taskAttemptId.getTaskId().getJobId());
-      
+
       GetDiagnosticsResponse response = recordFactory.newRecordInstance(GetDiagnosticsResponse.class);
       response.addAllDiagnostics(job.getTask(taskAttemptId.getTaskId()).getAttempt(taskAttemptId).getDiagnostics());
       return response;
     }
 
-    @Override 
+    @Override
     public FailTaskAttemptResponse failTaskAttempt(FailTaskAttemptRequest request) throws YarnRemoteException {
       throw RPCUtil.getRemoteException("Invalid operation on completed job");
     }
@@ -269,7 +268,7 @@ public class HistoryClientService extends AbstractService {
     public GetTaskReportsResponse getTaskReports(GetTaskReportsRequest request) throws YarnRemoteException {
       JobId jobId = request.getJobId();
       TaskType taskType = request.getTaskType();
-      
+
       GetTaskReportsResponse response = recordFactory.newRecordInstance(GetTaskReportsResponse.class);
       Job job = verifyAndGetJob(jobId);
       Collection<Task> tasks = job.getTasks(taskType).values();

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

@@ -21,7 +21,7 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.initID;
 
-import org.apache.hadoop.util.VersionInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.view.InfoBlock;
 
@@ -45,8 +45,9 @@ public class HsAboutPage extends HsView {
    * @return AttemptsBlock.class
    */
   @Override protected Class<? extends SubView> content() {
+    HistoryInfo info = new HistoryInfo();
     info("History Server").
-      _("BuildVersion", VersionInfo.getBuildVersion());
+      _("BuildVersion", info.getHadoopBuildVersion() + " on " + info.getHadoopVersionBuiltOn());
     return InfoBlock.class;
   }
 }

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

@@ -34,6 +34,9 @@ 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.Task;
 import org.apache.hadoop.mapreduce.v2.app.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfEntryInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
 import org.apache.hadoop.security.authorize.AccessControlList;
@@ -56,19 +59,6 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.*;
 public class HsJobBlock extends HtmlBlock {
   final AppContext appContext;
 
-  int killedMapAttempts = 0;
-  int failedMapAttempts = 0;
-  int successfulMapAttempts = 0;
-  int killedReduceAttempts = 0;
-  int failedReduceAttempts = 0;
-  int successfulReduceAttempts = 0;
-  long avgMapTime = 0;
-  long avgReduceTime = 0;
-  long avgShuffleTime = 0;
-  long avgSortTime = 0;
-  int numMaps;
-  int numReduces;
-
   @Inject HsJobBlock(AppContext appctx) {
     appContext = appctx;
   }
@@ -85,37 +75,30 @@ public class HsJobBlock extends HtmlBlock {
       return;
     }
     JobId jobID = MRApps.toJobID(jid);
-    Job job = appContext.getJob(jobID);
-    if (job == null) {
+    Job j = appContext.getJob(jobID);
+    if (j == null) {
       html.
         p()._("Sorry, ", jid, " not found.")._();
       return;
     }
-    Map<JobACL, AccessControlList> acls = job.getJobACLs();
-    List<AMInfo> amInfos = job.getAMInfos();
-    JobReport jobReport = job.getReport();
-    int mapTasks = job.getTotalMaps();
-    int mapTasksComplete = job.getCompletedMaps();
-    int reduceTasks = job.getTotalReduces();
-    int reducesTasksComplete = job.getCompletedReduces();
-    long startTime = jobReport.getStartTime();
-    long finishTime = jobReport.getFinishTime();
-    countTasksAndAttempts(job);
+    List<AMInfo> amInfos = j.getAMInfos();
+    JobInfo job = new JobInfo(j);
     ResponseInfo infoBlock = info("Job Overview").
         _("Job Name:", job.getName()).
         _("User Name:", job.getUserName()).
         _("Queue:", job.getQueueName()).
         _("State:", job.getState()).
         _("Uberized:", job.isUber()).
-        _("Started:", new Date(startTime)).
-        _("Finished:", new Date(finishTime)).
+        _("Started:", new Date(job.getStartTime())).
+        _("Finished:", new Date(job.getFinishTime())).
         _("Elapsed:", StringUtils.formatTime(
-            Times.elapsed(startTime, finishTime, false)));
+            Times.elapsed(job.getStartTime(), job.getFinishTime(), false)));
     
     String amString =
         amInfos.size() == 1 ? "ApplicationMaster" : "ApplicationMasters"; 
     
-    List<String> diagnostics = job.getDiagnostics();
+    // todo - switch to use JobInfo
+    List<String> diagnostics = j.getDiagnostics();
     if(diagnostics != null && !diagnostics.isEmpty()) {
       StringBuffer b = new StringBuffer();
       for(String diag: diagnostics) {
@@ -124,18 +107,17 @@ public class HsJobBlock extends HtmlBlock {
       infoBlock._("Diagnostics:", b.toString());
     }
 
-    if(numMaps > 0) {
-      infoBlock._("Average Map Time", StringUtils.formatTime(avgMapTime));
+    if(job.getNumMaps() > 0) {
+      infoBlock._("Average Map Time", StringUtils.formatTime(job.getAvgMapTime()));
     }
-    if(numReduces > 0) {
-      infoBlock._("Average Reduce Time", StringUtils.formatTime(avgReduceTime));
-      infoBlock._("Average Shuffle Time", StringUtils.formatTime(avgShuffleTime));
-      infoBlock._("Average Merge Time", StringUtils.formatTime(avgSortTime));
+    if(job.getNumReduces() > 0) {
+      infoBlock._("Average Reduce Time", StringUtils.formatTime(job.getAvgReduceTime()));
+      infoBlock._("Average Shuffle Time", StringUtils.formatTime(job.getAvgShuffleTime()));
+      infoBlock._("Average Merge Time", StringUtils.formatTime(job.getAvgMergeTime()));
     }
 
-    for(Map.Entry<JobACL, AccessControlList> entry : acls.entrySet()) {
-      infoBlock._("ACL "+entry.getKey().getAclName()+":",
-          entry.getValue().getAclString());
+    for (ConfEntryInfo entry : job.getAcls()) {
+      infoBlock._("ACL "+entry.getName()+":", entry.getValue());
     }
     DIV<Hamlet> div = html.
       _(InfoBlock.class).
@@ -154,18 +136,14 @@ public class HsJobBlock extends HtmlBlock {
             th(_TH, "Logs").
             _();
           for (AMInfo amInfo : amInfos) {
-            String nodeHttpAddress = amInfo.getNodeManagerHost() + 
-                ":" + amInfo.getNodeManagerHttpPort();
-            NodeId nodeId = BuilderUtils.newNodeId(
-                amInfo.getNodeManagerHost(), amInfo.getNodeManagerPort());
-            
+            AMAttemptInfo attempt = new AMAttemptInfo(amInfo,
+                job.getId(), job.getUserName(), "", "");
             table.tr().
-              td(String.valueOf(amInfo.getAppAttemptId().getAttemptId())).
-              td(new Date(amInfo.getStartTime()).toString()).
-              td().a(".nodelink", url("http://", nodeHttpAddress), 
-                  nodeHttpAddress)._().
-              td().a(".logslink", url("logs", nodeId.toString(), 
-                  amInfo.getContainerId().toString(), jid, job.getUserName()), 
+              td(String.valueOf(attempt.getAttemptId())).
+              td(new Date(attempt.getStartTime()).toString()).
+              td().a(".nodelink", url("http://", attempt.getNodeHttpAddress()), 
+                  attempt.getNodeHttpAddress())._().
+              td().a(".logslink", url(attempt.getShortLogsLink()), 
                       "logs")._().
             _();
           }
@@ -184,13 +162,13 @@ public class HsJobBlock extends HtmlBlock {
           tr(_ODD).
             th().
               a(url("tasks", jid, "m"), "Map")._().
-            td(String.valueOf(mapTasks)).
-            td(String.valueOf(mapTasksComplete))._().
+            td(String.valueOf(String.valueOf(job.getMapsTotal()))).
+            td(String.valueOf(String.valueOf(job.getMapsCompleted())))._().
           tr(_EVEN).
             th().
               a(url("tasks", jid, "r"), "Reduce")._().
-            td(String.valueOf(reduceTasks)).
-            td(String.valueOf(reducesTasksComplete))._()
+            td(String.valueOf(String.valueOf(job.getReducesTotal()))).
+            td(String.valueOf(String.valueOf(job.getReducesCompleted())))._()
           ._().
 
         // Attempts table
@@ -204,99 +182,27 @@ public class HsJobBlock extends HtmlBlock {
           th("Maps").
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(failedMapAttempts))._().
+              String.valueOf(job.getFailedMapAttempts()))._().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(killedMapAttempts))._().
+              String.valueOf(job.getKilledMapAttempts()))._().
           td().a(url("attempts", jid, "m",
               TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(successfulMapAttempts))._().
+              String.valueOf(job.getSuccessfulMapAttempts()))._().
         _().
         tr(_EVEN).
           th("Reduces").
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.FAILED.toString()), 
-              String.valueOf(failedReduceAttempts))._().
+              String.valueOf(job.getFailedReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.KILLED.toString()), 
-              String.valueOf(killedReduceAttempts))._().
+              String.valueOf(job.getKilledReduceAttempts()))._().
           td().a(url("attempts", jid, "r",
               TaskAttemptStateUI.SUCCESSFUL.toString()), 
-              String.valueOf(successfulReduceAttempts))._().
+              String.valueOf(job.getSuccessfulReduceAttempts()))._().
          _().
        _().
      _();
   }
-
-  /**
-   * Go through a job and update the member variables with counts for
-   * information to output in the page.
-   * @param job the job to get counts for.
-   */
-  private void countTasksAndAttempts(Job job) {
-    numReduces = 0;
-    numMaps = 0;
-    Map<TaskId, Task> tasks = job.getTasks();
-    for (Task task : tasks.values()) {
-      // Attempts counts
-      Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
-      for (TaskAttempt attempt : attempts.values()) {
-
-        int successful = 0, failed = 0, killed =0;
-
-        if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
-          //Do Nothing
-        } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt
-            .getState())) {
-          //Do Nothing
-        } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
-            .getState())) {
-          ++successful;
-        } else if (TaskAttemptStateUI.FAILED
-            .correspondsTo(attempt.getState())) {
-          ++failed;
-        } else if (TaskAttemptStateUI.KILLED
-            .correspondsTo(attempt.getState())) {
-          ++killed;
-        }
-
-        switch (task.getType()) {
-        case MAP:
-          successfulMapAttempts += successful;
-          failedMapAttempts += failed;
-          killedMapAttempts += killed;
-          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
-            numMaps++;
-            avgMapTime += (attempt.getFinishTime() -
-                attempt.getLaunchTime());
-          }
-          break;
-        case REDUCE:
-          successfulReduceAttempts += successful;
-          failedReduceAttempts += failed;
-          killedReduceAttempts += killed;
-          if(attempt.getState() == TaskAttemptState.SUCCEEDED) {
-            numReduces++;
-            avgShuffleTime += (attempt.getShuffleFinishTime() - 
-                attempt.getLaunchTime());
-            avgSortTime += attempt.getSortFinishTime() - 
-                attempt.getLaunchTime();
-            avgReduceTime += (attempt.getFinishTime() -
-                attempt.getShuffleFinishTime());
-          }
-          break;
-        }
-      }
-    }
-
-    if(numMaps > 0) {
-      avgMapTime = avgMapTime / numMaps;
-    }
-    
-    if(numReduces > 0) {
-      avgReduceTime = avgReduceTime / numReduces;
-      avgShuffleTime = avgShuffleTime / numReduces;
-      avgSortTime = avgSortTime / numReduces;
-    }
-  }
 }

+ 14 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsJobsBlock.java

@@ -21,10 +21,9 @@ package org.apache.hadoop.mapreduce.v2.hs.webapp;
 import java.text.SimpleDateFormat;
 import java.util.Date;
 
-import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 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.mapreduce.v2.hs.webapp.dao.JobInfo;
 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;
@@ -38,8 +37,8 @@ import com.google.inject.Inject;
  */
 public class HsJobsBlock extends HtmlBlock {
   final AppContext appContext;
-  static final SimpleDateFormat dateFormat = 
-    new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z"); 
+  static final SimpleDateFormat dateFormat =
+    new SimpleDateFormat("yyyy.MM.dd HH:mm:ss z");
 
   @Inject HsJobsBlock(AppContext appCtx) {
     appContext = appCtx;
@@ -68,28 +67,21 @@ public class HsJobsBlock extends HtmlBlock {
             th("Reduces Completed")._()._().
         tbody();
     LOG.info("Getting list of all Jobs.");
-    for (Job job : appContext.getAllJobs().values()) {
-      String jobID = MRApps.toString(job.getID());
-      JobReport report = job.getReport();
-      String mapsTotal = String.valueOf(job.getTotalMaps());
-      String mapsCompleted = String.valueOf(job.getCompletedMaps());
-      String reduceTotal = String.valueOf(job.getTotalReduces());
-      String reduceCompleted = String.valueOf(job.getCompletedReduces());
-      long startTime = report.getStartTime();
-      long finishTime = report.getFinishTime();
+    for (Job j : appContext.getAllJobs().values()) {
+      JobInfo job = new JobInfo(j);
       tbody.
         tr().
-          td(dateFormat.format(new Date(startTime))).
-          td(dateFormat.format(new Date(finishTime))).
-          td().a(url("job", jobID), jobID)._().
-          td(job.getName().toString()).
+          td(dateFormat.format(new Date(job.getStartTime()))).
+          td(dateFormat.format(new Date(job.getFinishTime()))).
+          td().a(url("job", job.getId()), job.getId())._().
+          td(job.getName()).
           td(job.getUserName()).
           td(job.getQueueName()).
-          td(job.getState().toString()).
-          td(mapsTotal).
-          td(mapsCompleted).
-          td(reduceTotal).
-          td(reduceCompleted)._();
+          td(job.getState()).
+          td(String.valueOf(job.getMapsTotal())).
+          td(String.valueOf(job.getMapsCompleted())).
+          td(String.valueOf(job.getReducesTotal())).
+          td(String.valueOf(job.getReducesCompleted()))._();
     }
     tbody._().
     tfoot().

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

@@ -20,12 +20,13 @@ 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.app.webapp.dao.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.util.Times;
@@ -65,7 +66,7 @@ public class HsTasksBlock extends HtmlBlock {
     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;
@@ -74,7 +75,7 @@ public class HsTasksBlock extends HtmlBlock {
       th().$colspan(attemptColSpan).$class("ui-state-default").
         _("Successful Attempt")._().
     _();
-    
+
     TR<THEAD<TABLE<Hamlet>>> theadRow = thead.
           tr().
             th("Name").
@@ -83,33 +84,33 @@ public class HsTasksBlock extends HtmlBlock {
             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);
-      
+      TaskInfo info = new TaskInfo(task);
+      String tid = info.getId();
+
+      long startTime = info.getStartTime();
+      long finishTime = info.getFinishTime();
+      long elapsed = info.getElapsedTime();
+
       long attemptStartTime = -1;
       long shuffleFinishTime = -1;
       long sortFinishTime = -1;
@@ -118,30 +119,31 @@ public class HsTasksBlock extends HtmlBlock {
       long elapsedSortTime = -1;;
       long elapsedReduceTime = -1;
       long attemptElapsed = -1;
-      TaskAttempt successful = getSuccessfulAttempt(task);
+      TaskAttempt successful = info.getSuccessful();
       if(successful != null) {
-        attemptStartTime = successful.getLaunchTime();
-        attemptFinishTime = successful.getFinishTime();
+        TaskAttemptInfo ta;
         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); 
+          ReduceTaskAttemptInfo rta = new ReduceTaskAttemptInfo(successful, type);
+          shuffleFinishTime = rta.getShuffleFinishTime();
+          sortFinishTime = rta.getMergeFinishTime();
+          elapsedShuffleTime = rta.getElapsedShuffleTime();
+          elapsedSortTime = rta.getElapsedMergeTime();
+          elapsedReduceTime = rta.getElapsedReduceTime();
+          ta = rta;
+        } else {
+          ta = new TaskAttemptInfo(successful, type, false);
         }
-        attemptElapsed =
-            Times.elapsed(attemptStartTime, attemptFinishTime, false);
+        attemptStartTime = ta.getStartTime();
+        attemptFinishTime = ta.getFinishTime();
+        attemptElapsed = ta.getElapsedTime();
       }
-      
+
       TR<TBODY<TABLE<Hamlet>>> row = tbody.tr();
       row.
           td().
-            br().$title(String.valueOf(task.getID().getId()))._(). // sorting
+            br().$title(String.valueOf(info.getTaskNum()))._(). // sorting
             a(url("task", tid), tid)._().
-          td(report.getTaskState().toString()).
+          td(info.getState()).
           td().
             br().$title(String.valueOf(startTime))._().
             _(Times.format(startTime))._().
@@ -166,7 +168,7 @@ public class HsTasksBlock extends HtmlBlock {
           td().
             br().$title(String.valueOf(attemptFinishTime))._().
             _(Times.format(attemptFinishTime))._();
-      
+
       if(type == TaskType.REDUCE) {
         row.td().
           br().$title(String.valueOf(elapsedShuffleTime))._().
@@ -178,7 +180,7 @@ public class HsTasksBlock extends HtmlBlock {
           br().$title(String.valueOf(elapsedReduceTime))._().
         _(formatTime(elapsedReduceTime))._();
       }
-      
+
       row.td().
         br().$title(String.valueOf(attemptElapsed))._().
         _(formatTime(attemptElapsed))._();
@@ -194,7 +196,7 @@ public class HsTasksBlock extends HtmlBlock {
         .$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")._()._();
@@ -216,20 +218,12 @@ public class HsTasksBlock extends HtmlBlock {
 
     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;
-  }
+
 }

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

@@ -27,6 +27,7 @@ import static org.apache.hadoop.yarn.webapp.YarnWebParams.NM_NODENAME;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMParams;
 import org.apache.hadoop.mapreduce.v2.hs.HistoryContext;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 
 public class HsWebApp extends WebApp implements AMParams {
@@ -39,6 +40,9 @@ public class HsWebApp extends WebApp implements AMParams {
 
   @Override
   public void setup() {
+    bind(HsWebServices.class);
+    bind(JAXBContextResolver.class);
+    bind(GenericExceptionHandler.class);
     bind(AppContext.class).toInstance(history);
     route("/", HsController.class);
     route("/app", HsController.class);

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

@@ -0,0 +1,469 @@
+/**
+ * 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 java.io.IOException;
+
+import javax.ws.rs.GET;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.UriInfo;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app.AppContext;
+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.dao.ConfInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.yarn.YarnException;
+import org.apache.hadoop.yarn.webapp.BadRequestException;
+import org.apache.hadoop.yarn.webapp.NotFoundException;
+import org.apache.hadoop.yarn.webapp.WebApp;
+
+import com.google.inject.Inject;
+
+@Path("/ws/v1/history")
+public class HsWebServices {
+  private final AppContext appCtx;
+  private WebApp webapp;
+  private final Configuration conf;
+
+  @Context
+  UriInfo uriInfo;
+
+  @Inject
+  public HsWebServices(final AppContext appCtx, final Configuration conf,
+      final WebApp webapp) {
+    this.appCtx = appCtx;
+    this.conf = conf;
+    this.webapp = webapp;
+  }
+
+  @GET
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public HistoryInfo get() {
+    return getHistoryInfo();
+  }
+
+  @GET
+  @Path("/info")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public HistoryInfo getHistoryInfo() {
+    return new HistoryInfo();
+  }
+
+  @GET
+  @Path("/mapreduce/jobs")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobsInfo getJobs(@QueryParam("user") String userQuery,
+      @QueryParam("limit") String count,
+      @QueryParam("queue") String queueQuery,
+      @QueryParam("startedTimeBegin") String startedBegin,
+      @QueryParam("startedTimeEnd") String startedEnd,
+      @QueryParam("finishedTimeBegin") String finishBegin,
+      @QueryParam("finishedTimeEnd") String finishEnd) {
+    JobsInfo allJobs = new JobsInfo();
+    long num = 0;
+    boolean checkCount = false;
+    boolean checkStart = false;
+    boolean checkEnd = false;
+    long countNum = 0;
+
+    // set values suitable in case both of begin/end not specified
+    long sBegin = 0;
+    long sEnd = Long.MAX_VALUE;
+    long fBegin = 0;
+    long fEnd = Long.MAX_VALUE;
+
+    if (count != null && !count.isEmpty()) {
+      checkCount = true;
+      try {
+        countNum = Long.parseLong(count);
+      } catch (NumberFormatException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+      if (countNum <= 0) {
+        throw new BadRequestException("limit value must be greater then 0");
+      }
+    }
+
+    if (startedBegin != null && !startedBegin.isEmpty()) {
+      checkStart = true;
+      try {
+        sBegin = Long.parseLong(startedBegin);
+      } catch (NumberFormatException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+      if (sBegin < 0) {
+        throw new BadRequestException("startedTimeBegin must be greater than 0");
+      }
+    }
+    if (startedEnd != null && !startedEnd.isEmpty()) {
+      checkStart = true;
+      try {
+        sEnd = Long.parseLong(startedEnd);
+      } catch (NumberFormatException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+      if (sEnd < 0) {
+        throw new BadRequestException("startedTimeEnd must be greater than 0");
+      }
+    }
+    if (sBegin > sEnd) {
+      throw new BadRequestException(
+          "startedTimeEnd must be greater than startTimeBegin");
+    }
+
+    if (finishBegin != null && !finishBegin.isEmpty()) {
+      checkEnd = true;
+      try {
+        fBegin = Long.parseLong(finishBegin);
+      } catch (NumberFormatException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+      if (fBegin < 0) {
+        throw new BadRequestException("finishTimeBegin must be greater than 0");
+      }
+    }
+    if (finishEnd != null && !finishEnd.isEmpty()) {
+      checkEnd = true;
+      try {
+        fEnd = Long.parseLong(finishEnd);
+      } catch (NumberFormatException e) {
+        throw new BadRequestException(e.getMessage());
+      }
+      if (fEnd < 0) {
+        throw new BadRequestException("finishTimeEnd must be greater than 0");
+      }
+    }
+    if (fBegin > fEnd) {
+      throw new BadRequestException(
+          "finishTimeEnd must be greater than finishTimeBegin");
+    }
+
+    for (Job job : appCtx.getAllJobs().values()) {
+      if (checkCount && num == countNum) {
+        break;
+      }
+
+      // getAllJobs only gives you a partial we want a full
+      Job fullJob = appCtx.getJob(job.getID());
+      if (fullJob == null) {
+        continue;
+      }
+
+      JobInfo jobInfo = new JobInfo(fullJob);
+      // can't really validate queue is a valid one since queues could change
+      if (queueQuery != null && !queueQuery.isEmpty()) {
+        if (!jobInfo.getQueueName().equals(queueQuery)) {
+          continue;
+        }
+      }
+
+      if (userQuery != null && !userQuery.isEmpty()) {
+        if (!jobInfo.getName().equals(userQuery)) {
+          continue;
+        }
+      }
+
+      if (checkStart
+          && (jobInfo.getStartTime() < sBegin || jobInfo.getStartTime() > sEnd)) {
+        continue;
+      }
+      if (checkEnd
+          && (jobInfo.getFinishTime() < fBegin || jobInfo.getFinishTime() > fEnd)) {
+        continue;
+      }
+
+      allJobs.add(jobInfo);
+      num++;
+    }
+    return allJobs;
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobInfo getJob(@PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    return new JobInfo(job);
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/attempts")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public AMAttemptsInfo getJobAttempts(@PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    AMAttemptsInfo amAttempts = new AMAttemptsInfo();
+    for (AMInfo amInfo : job.getAMInfos()) {
+      AMAttemptInfo attempt = new AMAttemptInfo(amInfo, MRApps.toString(job
+          .getID()), job.getUserName(), uriInfo.getBaseUri().toString(),
+          webapp.name());
+      amAttempts.add(attempt);
+    }
+    return amAttempts;
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobCounterInfo getJobCounters(@PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    return new JobCounterInfo(this.appCtx, job);
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobTaskCounterInfo getSingleTaskCounters(
+      @PathParam("jobid") String jid, @PathParam("taskid") String tid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = this.appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    return new JobTaskCounterInfo(task);
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/conf")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public ConfInfo getJobConf(@PathParam("jobid") String jid) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    ConfInfo info;
+    try {
+      info = new ConfInfo(job, this.conf);
+    } catch (IOException e) {
+      throw new NotFoundException("unable to load configuration for job: "
+          + jid);
+    }
+
+    return info;
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TasksInfo getJobTasks(@PathParam("jobid") String jid,
+      @QueryParam("type") String type) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TasksInfo allTasks = new TasksInfo();
+    for (Task task : job.getTasks().values()) {
+      TaskType ttype = null;
+      if (type != null && !type.isEmpty()) {
+        try {
+          ttype = MRApps.taskType(type);
+        } catch (YarnException e) {
+          throw new BadRequestException("tasktype must be either m or r");
+        }
+      }
+      if (ttype != null && task.getType() != ttype) {
+        continue;
+      }
+      allTasks.add(new TaskInfo(task));
+    }
+    return allTasks;
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskInfo getJobTask(@PathParam("jobid") String jid,
+      @PathParam("taskid") String tid) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    return new TaskInfo(task);
+
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskAttemptsInfo getJobTaskAttempts(@PathParam("jobid") String jid,
+      @PathParam("taskid") String tid) {
+    TaskAttemptsInfo attempts = new TaskAttemptsInfo();
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    for (TaskAttempt ta : task.getAttempts().values()) {
+      if (ta != null) {
+        if (task.getType() == TaskType.REDUCE) {
+          attempts.add(new ReduceTaskAttemptInfo(ta, task.getType()));
+        } else {
+          attempts.add(new TaskAttemptInfo(ta, task.getType(), false));
+        }
+      }
+    }
+    return attempts;
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public TaskAttemptInfo getJobTaskAttemptId(@PathParam("jobid") String jid,
+      @PathParam("taskid") String tid, @PathParam("attemptid") String attId) {
+    Job job = this.appCtx.getJob(MRApps.toJobID(jid));
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
+    if (attemptId == null) {
+      throw new NotFoundException("task attempt id " + attId
+          + " not found or invalid");
+    }
+    TaskAttempt ta = task.getAttempt(attemptId);
+    if (ta == null) {
+      throw new NotFoundException("Error getting info on task attempt id "
+          + attId);
+    }
+    if (task.getType() == TaskType.REDUCE) {
+      return new ReduceTaskAttemptInfo(ta, task.getType());
+    } else {
+      return new TaskAttemptInfo(ta, task.getType(), false);
+    }
+  }
+
+  @GET
+  @Path("/mapreduce/jobs/{jobid}/tasks/{taskid}/attempts/{attemptid}/counters")
+  @Produces({ MediaType.APPLICATION_JSON, MediaType.APPLICATION_XML })
+  public JobTaskAttemptCounterInfo getJobTaskAttemptIdCounters(
+      @PathParam("jobid") String jid, @PathParam("taskid") String tid,
+      @PathParam("attemptid") String attId) {
+    JobId jobId = MRApps.toJobID(jid);
+    if (jobId == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    Job job = this.appCtx.getJob(jobId);
+    if (job == null) {
+      throw new NotFoundException("job, " + jid + ", is not found");
+    }
+    TaskId taskID = MRApps.toTaskID(tid);
+    if (taskID == null) {
+      throw new NotFoundException("taskid " + tid + " not found or invalid");
+    }
+    Task task = job.getTask(taskID);
+    if (task == null) {
+      throw new NotFoundException("task not found with id " + tid);
+    }
+    TaskAttemptId attemptId = MRApps.toTaskAttemptID(attId);
+    if (attemptId == null) {
+      throw new NotFoundException("task attempt id " + attId
+          + " not found or invalid");
+    }
+    TaskAttempt ta = task.getAttempt(attemptId);
+    if (ta == null) {
+      throw new NotFoundException("Error getting info on task attempt id "
+          + attId);
+    }
+    return new JobTaskAttemptCounterInfo(ta);
+  }
+
+}

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

@@ -0,0 +1,78 @@
+/**
+ * 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 com.google.inject.Singleton;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.ws.rs.ext.ContextResolver;
+import javax.ws.rs.ext.Provider;
+import javax.xml.bind.JAXBContext;
+
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ConfInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterGroupInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.CounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskAttemptCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.JobTaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.ReduceTaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterGroupInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TaskCounterInfo;
+import org.apache.hadoop.mapreduce.v2.app.webapp.dao.TasksInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.AMAttemptsInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.HistoryInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobInfo;
+import org.apache.hadoop.mapreduce.v2.hs.webapp.dao.JobsInfo;
+
+@Singleton
+@Provider
+public class JAXBContextResolver implements ContextResolver<JAXBContext> {
+
+  private JAXBContext context;
+  private final Set<Class> types;
+
+  // you have to specify all the dao classes here
+  private final Class[] cTypes = { HistoryInfo.class, JobInfo.class,
+      JobsInfo.class, TasksInfo.class, TaskAttemptsInfo.class, ConfInfo.class,
+      CounterInfo.class, JobTaskCounterInfo.class,
+      JobTaskAttemptCounterInfo.class, 
+      TaskCounterInfo.class, JobCounterInfo.class, ReduceTaskAttemptInfo.class,
+      TaskAttemptInfo.class, TaskAttemptsInfo.class, CounterGroupInfo.class,
+      TaskCounterGroupInfo.class, 
+      AMAttemptInfo.class, AMAttemptsInfo.class};
+
+  public JAXBContextResolver() throws Exception {
+    this.types = new HashSet<Class>(Arrays.asList(cTypes));
+    this.context = new JSONJAXBContext(JSONConfiguration.natural()
+        .rootUnwrapping(false).build(), cTypes);
+  }
+
+  @Override
+  public JAXBContext getContext(Class<?> objectType) {
+    return (types.contains(objectType)) ? context : null;
+  }
+}

+ 96 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptInfo.java

@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.mapreduce.v2.hs.webapp.dao;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.util.StringHelper.ujoin;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.v2.api.records.AMInfo;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+
+@XmlRootElement(name = "amAttempt")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AMAttemptInfo {
+
+  protected String nodeHttpAddress;
+  protected String nodeId;
+  protected int id;
+  protected long startTime;
+  protected String containerId;
+  protected String logsLink;
+
+  @XmlTransient
+  protected String shortLogsLink;
+
+  public AMAttemptInfo() {
+  }
+
+  public AMAttemptInfo(AMInfo amInfo, String jobId, String user, String host,
+      String pathPrefix) {
+    this.nodeHttpAddress = amInfo.getNodeManagerHost() + ":"
+        + amInfo.getNodeManagerHttpPort();
+    NodeId nodeId = BuilderUtils.newNodeId(amInfo.getNodeManagerHost(),
+        amInfo.getNodeManagerPort());
+    this.nodeId = nodeId.toString();
+    this.id = amInfo.getAppAttemptId().getAttemptId();
+    this.startTime = amInfo.getStartTime();
+    this.containerId = amInfo.getContainerId().toString();
+    this.logsLink = join(
+        host,
+        pathPrefix,
+        ujoin("logs", nodeId.toString(), amInfo.getContainerId().toString(),
+            jobId, user));
+    this.shortLogsLink = ujoin("logs", nodeId.toString(), amInfo
+        .getContainerId().toString(), jobId, user);
+  }
+
+  public String getNodeHttpAddress() {
+    return this.nodeHttpAddress;
+  }
+
+  public String getNodeId() {
+    return this.nodeId;
+  }
+
+  public int getAttemptId() {
+    return this.id;
+  }
+
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public String getContainerId() {
+    return this.containerId;
+  }
+
+  public String getLogsLink() {
+    return this.logsLink;
+  }
+
+  public String getShortLogsLink() {
+    return this.shortLogsLink;
+  }
+
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/AMAttemptsInfo.java

@@ -0,0 +1,43 @@
+/**
+ * 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 joblicable 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "attempts")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AMAttemptsInfo {
+
+  protected ArrayList<AMAttemptInfo> attempt = new ArrayList<AMAttemptInfo>();
+
+  public AMAttemptsInfo() {
+  } // JAXB needs this
+
+  public void add(AMAttemptInfo info) {
+    this.attempt.add(info);
+  }
+
+  public ArrayList<AMAttemptInfo> getAttempts() {
+    return this.attempt;
+  }
+
+}

+ 53 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/HistoryInfo.java

@@ -0,0 +1,53 @@
+/**
+ * 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.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.util.VersionInfo;
+
+@XmlRootElement
+@XmlAccessorType(XmlAccessType.FIELD)
+public class HistoryInfo {
+
+  protected String hadoopVersion;
+  protected String hadoopBuildVersion;
+  protected String hadoopVersionBuiltOn;
+
+  public HistoryInfo() {
+    this.hadoopVersion = VersionInfo.getVersion();
+    this.hadoopBuildVersion = VersionInfo.getBuildVersion();
+    this.hadoopVersionBuiltOn = VersionInfo.getDate();
+  }
+
+  public String getHadoopVersion() {
+    return this.hadoopVersion;
+  }
+
+  public String getHadoopBuildVersion() {
+    return this.hadoopBuildVersion;
+  }
+
+  public String getHadoopVersionBuiltOn() {
+    return this.hadoopVersionBuiltOn;
+  }
+
+}

+ 295 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobInfo.java

@@ -0,0 +1,295 @@
+/**
+ * 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.dao;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlTransient;
+
+import org.apache.hadoop.mapreduce.JobACL;
+import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
+import org.apache.hadoop.mapreduce.v2.app.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.dao.ConfEntryInfo;
+import org.apache.hadoop.mapreduce.v2.hs.CompletedJob;
+import org.apache.hadoop.mapreduce.v2.util.MRApps;
+import org.apache.hadoop.mapreduce.v2.util.MRApps.TaskAttemptStateUI;
+import org.apache.hadoop.security.authorize.AccessControlList;
+
+@XmlRootElement(name = "job")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobInfo {
+
+  protected long startTime;
+  protected long finishTime;
+  protected String id;
+  protected String name;
+  protected String queue;
+  protected String user;
+  protected String state;
+  protected int mapsTotal;
+  protected int mapsCompleted;
+  protected int reducesTotal;
+  protected int reducesCompleted;
+  protected boolean uberized;
+  protected String diagnostics;
+  protected long avgMapTime = 0;
+  protected long avgReduceTime = 0;
+  protected long avgShuffleTime = 0;
+  protected long avgMergeTime = 0;
+  protected int failedReduceAttempts = 0;
+  protected int killedReduceAttempts = 0;
+  protected int successfulReduceAttempts = 0;
+  protected int failedMapAttempts = 0;
+  protected int killedMapAttempts = 0;
+  protected int successfulMapAttempts = 0;
+  protected ArrayList<ConfEntryInfo> acls;
+
+  @XmlTransient
+  protected int numMaps;
+  @XmlTransient
+  protected int numReduces;
+
+  public JobInfo() {
+  }
+
+  public JobInfo(Job job) {
+    this.id = MRApps.toString(job.getID());
+    JobReport report = job.getReport();
+    countTasksAndAttempts(job);
+    this.mapsTotal = job.getTotalMaps();
+    this.mapsCompleted = job.getCompletedMaps();
+    this.reducesTotal = job.getTotalReduces();
+    this.reducesCompleted = job.getCompletedReduces();
+    this.startTime = report.getStartTime();
+    this.finishTime = report.getFinishTime();
+    this.name = job.getName().toString();
+    this.queue = job.getQueueName();
+    this.user = job.getUserName();
+    this.state = job.getState().toString();
+    this.uberized = job.isUber();
+    List<String> diagnostics = job.getDiagnostics();
+    if (diagnostics != null && !diagnostics.isEmpty()) {
+      StringBuffer b = new StringBuffer();
+      for (String diag : diagnostics) {
+        b.append(diag);
+      }
+      this.diagnostics = b.toString();
+    }
+
+    this.acls = new ArrayList<ConfEntryInfo>();
+    if (job instanceof CompletedJob) {
+      Map<JobACL, AccessControlList> allacls = job.getJobACLs();
+      if (allacls != null) {
+        for (Map.Entry<JobACL, AccessControlList> entry : allacls.entrySet()) {
+          this.acls.add(new ConfEntryInfo(entry.getKey().getAclName(), entry
+              .getValue().getAclString()));
+        }
+      }
+    }
+  }
+
+  public long getNumMaps() {
+    return numMaps;
+  }
+
+  public long getNumReduces() {
+    return numReduces;
+  }
+
+  public long getAvgMapTime() {
+    return avgMapTime;
+  }
+
+  public long getAvgReduceTime() {
+    return avgReduceTime;
+  }
+
+  public long getAvgShuffleTime() {
+    return avgShuffleTime;
+  }
+
+  public long getAvgMergeTime() {
+    return avgMergeTime;
+  }
+
+  public long getFailedReduceAttempts() {
+    return failedReduceAttempts;
+  }
+
+  public long getKilledReduceAttempts() {
+    return killedReduceAttempts;
+  }
+
+  public long getSuccessfulReduceAttempts() {
+    return successfulReduceAttempts;
+  }
+
+  public long getFailedMapAttempts() {
+    return failedMapAttempts;
+  }
+
+  public long getKilledMapAttempts() {
+    return killedMapAttempts;
+  }
+
+  public long getSuccessfulMapAttempts() {
+    return successfulMapAttempts;
+  }
+
+  public ArrayList<ConfEntryInfo> getAcls() {
+    return acls;
+  }
+
+  public int getReducesCompleted() {
+    return this.reducesCompleted;
+  }
+
+  public int getReducesTotal() {
+    return this.reducesTotal;
+  }
+
+  public int getMapsCompleted() {
+    return this.mapsCompleted;
+  }
+
+  public int getMapsTotal() {
+    return this.mapsTotal;
+  }
+
+  public String getState() {
+    return this.state;
+  }
+
+  public String getUserName() {
+    return this.user;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  public String getQueueName() {
+    return this.queue;
+  }
+
+  public String getId() {
+    return this.id;
+  }
+
+  public long getStartTime() {
+    return this.startTime;
+  }
+
+  public long getFinishTime() {
+    return this.finishTime;
+  }
+
+  public boolean isUber() {
+    return this.uberized;
+  }
+
+  public String getDiagnostics() {
+    return this.diagnostics;
+  }
+
+  /**
+   * Go through a job and update the member variables with counts for
+   * information to output in the page.
+   *
+   * @param job
+   *          the job to get counts for.
+   */
+  private void countTasksAndAttempts(Job job) {
+    numReduces = 0;
+    numMaps = 0;
+    final Map<TaskId, Task> tasks = job.getTasks();
+    if (tasks == null) {
+      return;
+    }
+    for (Task task : tasks.values()) {
+      // Attempts counts
+      Map<TaskAttemptId, TaskAttempt> attempts = task.getAttempts();
+      int successful, failed, killed;
+      for (TaskAttempt attempt : attempts.values()) {
+
+        successful = 0;
+        failed = 0;
+        killed = 0;
+        if (TaskAttemptStateUI.NEW.correspondsTo(attempt.getState())) {
+          // Do Nothing
+        } else if (TaskAttemptStateUI.RUNNING.correspondsTo(attempt.getState())) {
+          // Do Nothing
+        } else if (TaskAttemptStateUI.SUCCESSFUL.correspondsTo(attempt
+            .getState())) {
+          ++successful;
+        } else if (TaskAttemptStateUI.FAILED.correspondsTo(attempt.getState())) {
+          ++failed;
+        } else if (TaskAttemptStateUI.KILLED.correspondsTo(attempt.getState())) {
+          ++killed;
+        }
+
+        switch (task.getType()) {
+        case MAP:
+          successfulMapAttempts += successful;
+          failedMapAttempts += failed;
+          killedMapAttempts += killed;
+          if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numMaps++;
+            avgMapTime += (attempt.getFinishTime() - attempt.getLaunchTime());
+          }
+          break;
+        case REDUCE:
+          successfulReduceAttempts += successful;
+          failedReduceAttempts += failed;
+          killedReduceAttempts += killed;
+          if (attempt.getState() == TaskAttemptState.SUCCEEDED) {
+            numReduces++;
+            avgShuffleTime += (attempt.getShuffleFinishTime() - attempt
+                .getLaunchTime());
+            avgMergeTime += attempt.getSortFinishTime()
+                - attempt.getLaunchTime();
+            avgReduceTime += (attempt.getFinishTime() - attempt
+                .getShuffleFinishTime());
+          }
+          break;
+        }
+      }
+    }
+
+    if (numMaps > 0) {
+      avgMapTime = avgMapTime / numMaps;
+    }
+
+    if (numReduces > 0) {
+      avgReduceTime = avgReduceTime / numReduces;
+      avgShuffleTime = avgShuffleTime / numReduces;
+      avgMergeTime = avgMergeTime / numReduces;
+    }
+  }
+
+}

+ 43 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/dao/JobsInfo.java

@@ -0,0 +1,43 @@
+/**
+ * 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 joblicable 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.dao;
+
+import java.util.ArrayList;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+@XmlRootElement(name = "jobs")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class JobsInfo {
+
+  protected ArrayList<JobInfo> job = new ArrayList<JobInfo>();
+
+  public JobsInfo() {
+  } // JAXB needs this
+
+  public void add(JobInfo jobInfo) {
+    this.job.add(jobInfo);
+  }
+
+  public ArrayList<JobInfo> getJobs() {
+    return this.job;
+  }
+
+}

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

@@ -168,7 +168,7 @@ public class TestClientServiceDelegate {
     GetJobReportResponse jobReportResponse1 = mock(GetJobReportResponse.class);
     when(jobReportResponse1.getJobReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "jobName-firstGen", "user",
-            JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null));
+            JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null, false));
 
     // First AM returns a report with jobName firstGen and simulates AM shutdown
     // on second invocation.
@@ -180,7 +180,7 @@ public class TestClientServiceDelegate {
     GetJobReportResponse jobReportResponse2 = mock(GetJobReportResponse.class);
     when(jobReportResponse2.getJobReport()).thenReturn(
         MRBuilderUtils.newJobReport(jobId, "jobName-secondGen", "user",
-            JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null));
+            JobState.RUNNING, 0, 0, 0, 0, 0, 0, 0, "anything", null, false));
 
     // Second AM generation returns a report with jobName secondGen
     MRClientProtocol secondGenAMProxy = mock(MRClientProtocol.class);

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

@@ -49,6 +49,7 @@ public class TestUberAM extends TestMRJobs {
   }
 
   @Override
+  @Test
   public void testSleepJob()
   throws IOException, InterruptedException, ClassNotFoundException {
     if (mrCluster != null) {
@@ -84,6 +85,7 @@ public class TestUberAM extends TestMRJobs {
   }
 
   @Override
+  @Test
   public void testRandomWriter()
   throws IOException, InterruptedException, ClassNotFoundException {
     super.testRandomWriter();
@@ -101,6 +103,7 @@ public class TestUberAM extends TestMRJobs {
   }
 
   @Override
+  @Test
   public void testFailingMapper()
   throws IOException, InterruptedException, ClassNotFoundException {
     LOG.info("\n\n\nStarting uberized testFailingMapper().");

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

@@ -238,7 +238,7 @@
     <dependency>
       <groupId>com.google.inject.extensions</groupId>
       <artifactId>guice-servlet</artifactId>
-      <version>2.0</version>
+      <version>3.0</version>
     </dependency>
     <dependency>
       <groupId>junit</groupId>

+ 0 - 27
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/AMRMProtocol.genavro

@@ -1,27 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol AMRMProtocol {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-  
-   // Scheduler
-   record Priority {
-     int priority;
-   }
-  
-   record ResourceRequest {
-    Priority priority;
-    string hostName;
-    Resource capability;
-    int numContainers;
-  }
-  record AMResponse {
-    boolean reboot;
-    int responseId;
-    array<Container> containers;
-  }
-  
-  void registerApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
-  void finishApplicationMaster(ApplicationMaster applicationMaster) throws YarnRemoteException;
-  AMResponse allocate(ApplicationStatus status, array<ResourceRequest> ask, array<Container> release) throws YarnRemoteException;
-  
- }

+ 0 - 45
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ClientRMProtocol.genavro

@@ -1,45 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ClientRMProtocol {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  record Priority {
-   int priority;
-  }
-
-  record ApplicationSubmissionContext {
-    ApplicationID applicationId;
-    union {null, string} applicationName;
-    Resource masterCapability; // TODO: Needs RM validation
-
-    //all the files required by the container to run the ApplicationMaster
-    //KEY-> destination dir name
-    //VALUE-> source path
-    map<URL> resources;
-    union {null, map<LocalResource>} resources_todo;
-    
-    // TODO - Remove fsTokens (url encoded)
-    union {null, array<string>} fsTokens;
-    union {null, bytes} fsTokens_todo;
-    
-    //env to be set before launching the command for ApplicationMaster
-    //KEY-> env variable name
-    //VALUE -> env variable value.
-    map<string> environment;
-    //command-line of the container that is going to launch the ApplicationMaster.
-    array<string> command;
-    union {null, string} queue;
-    union {null, Priority} priority;
-    string user; // TODO: Shouldn't pass it like this.
-  }
-  
-  record YarnClusterMetrics {
-    int numNodeManagers;
-  }
-
-  ApplicationID getNewApplicationId() throws YarnRemoteException;
-  ApplicationMaster getApplicationMaster(ApplicationID applicationId) throws YarnRemoteException;
-  void submitApplication(ApplicationSubmissionContext context) throws YarnRemoteException;
-  void finishApplication(ApplicationID applicationId) throws YarnRemoteException;
-  YarnClusterMetrics getClusterMetrics() throws YarnRemoteException;  
-}

+ 0 - 37
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/ContainerManager.genavro

@@ -1,37 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ContainerManager {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-
-  record ContainerLaunchContext {
-    ContainerID id;
-    string user; // TODO: Shouldn't pass it like this.
-    Resource resource; // TODO: Needs RM validation
-    union {null, map<LocalResource>} resources;
-
-    union {null, bytes} containerTokens; // FileSystem related and other application specific tokens.
-    union {null, map<bytes>} serviceData;
-
-    //env to be set before launching the command
-    //KEY-> env variable name
-    //VALUE -> env variable value.
-    map<string> env;
-
-    //commandline to launch the container. All resources are downloaded in the 
-    //working directory of the command.
-    array<string> command;
-  }
-
-  record ContainerStatus {
-    ContainerID containerID;
-    ContainerState state;
-    int exitStatus;
-  }
-
-  void startContainer(ContainerLaunchContext container) throws YarnRemoteException;
-  void stopContainer(ContainerID containerID) throws YarnRemoteException;
-  void cleanupContainer(ContainerID containerID) throws YarnRemoteException;
-
-  ContainerStatus getContainerStatus(ContainerID containerID) throws YarnRemoteException;
-
-}

+ 0 - 109
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/avro/yarn-types.genavro

@@ -1,109 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol types {
-
-  record ApplicationID {
-    int id;
-    long clusterTimeStamp;
-  }
-
-  record ContainerID {
-    ApplicationID appID; // the application id to which this container belong.
-    int id;// unique string for this application
-  }
-
-  error YarnRemoteException {
-    union { null, string } message;
-    union { null, string } trace; //stackTrace
-    union { null, YarnRemoteException } cause;
-  }
-  
-  record Resource {
-    int memory;
-    //int diskspace;
-  }
-
-  // State of the container on the ContainerManager.
-  enum ContainerState {
-    INTIALIZING,
-    RUNNING,
-    COMPLETE
-  }
-
-  record ContainerToken {
-    bytes identifier;
-    bytes password;
-    string kind;
-    string service;
-  }
-
-  record Container {
-    ContainerID id;
-    string hostName;
-    Resource resource;
-    ContainerState state;
-    union {ContainerToken, null} containerToken;
-  }
-
-  enum ApplicationState {
-    PENDING,
-    ALLOCATING,
-   	ALLOCATED,
-   	EXPIRED_PENDING,
-   	LAUNCHING,
-   	LAUNCHED,
-    RUNNING,
-    PAUSED,
-    CLEANUP,
-    COMPLETED,
-    KILLED,
-    FAILED
-  }
- 
- record ApplicationStatus {
-    int responseID; // TODO: This should be renamed as previousResponseID
-    ApplicationID applicationId;
-    float progress;
-    long lastSeen;
-  }
-  
-  record ApplicationMaster {
-    ApplicationID applicationId;
-    union { null, string } host;
-    int rpcPort;
-    int httpPort;
-    ApplicationStatus status;
-    ApplicationState state;
-    union { null, string } clientToken;
-  }
-
-  record URL {
-    string scheme;
-    union { null, string } host;
-    int port;
-    string file;
-  }
-
-  enum LocalResourceVisibility {
-    // accessible to applications from all users
-    PUBLIC,
-    // accessible only to applications from the submitting user
-    PRIVATE,
-    // accessible only to this application
-    APPLICATION
-  }
-
-  enum LocalResourceType {
-    // an archive to be expanded
-    ARCHIVE,
-    // uninterpreted file
-    FILE
-  }
-
-  record LocalResource {
-    URL resource;
-    long size;
-    long timestamp;
-    LocalResourceType type;
-    LocalResourceVisibility state;
-  }
-}

+ 0 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -24,7 +24,6 @@ import org.apache.hadoop.security.UserGroupInformation;
  * This is the API for the applications comprising of constants that YARN sets
  * up for the applications and the containers.
  * 
- * TODO: Should also be defined in avro/pb IDLs
  * TODO: Investigate the semantics and security of each cross-boundary refs.
  */
 public interface ApplicationConstants {

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

@@ -91,12 +91,7 @@ public class YarnConfiguration extends Configuration {
   public static final String RM_CLIENT_THREAD_COUNT =
     RM_PREFIX + "client.thread-count";
   public static final int DEFAULT_RM_CLIENT_THREAD_COUNT = 10;
-  
-  /** The expiry interval for application master reporting.*/
-  public static final String RM_AM_EXPIRY_INTERVAL_MS = 
-    RM_PREFIX  + "am.liveness-monitor.expiry-interval-ms";
-  public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
-  
+
   /** The Kerberos principal for the resource manager.*/
   public static final String RM_PRINCIPAL =
     RM_PREFIX + "principal";
@@ -126,7 +121,17 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_RM_RESOURCE_TRACKER_PORT = 8025;
   public static final String DEFAULT_RM_RESOURCE_TRACKER_ADDRESS =
     "0.0.0.0:" + DEFAULT_RM_RESOURCE_TRACKER_PORT;
-  
+
+  /** The expiry interval for application master reporting.*/
+  public static final String RM_AM_EXPIRY_INTERVAL_MS = 
+    YARN_PREFIX  + "am.liveness-monitor.expiry-interval-ms";
+  public static final int DEFAULT_RM_AM_EXPIRY_INTERVAL_MS = 600000;
+
+  /** How long to wait until a node manager is considered dead.*/
+  public static final String RM_NM_EXPIRY_INTERVAL_MS = 
+    YARN_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
+  public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
+
   /** Are acls enabled.*/
   public static final String YARN_ACL_ENABLE = 
     YARN_PREFIX + "acl.enable";
@@ -160,12 +165,7 @@ public class YarnConfiguration extends Configuration {
   /** The keytab for the resource manager.*/
   public static final String RM_KEYTAB = 
     RM_PREFIX + "keytab";
-  
-  /** How long to wait until a node manager is considered dead.*/
-  public static final String RM_NM_EXPIRY_INTERVAL_MS = 
-    RM_PREFIX + "nm.liveness-monitor.expiry-interval-ms";
-  public static final int DEFAULT_RM_NM_EXPIRY_INTERVAL_MS = 600000;
-  
+
   /** How long to wait until a container is considered dead.*/
   public static final String RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS = 
     RM_PREFIX + "rm.container-allocation.expiry-interval-ms";
@@ -293,10 +293,16 @@ public class YarnConfiguration extends Configuration {
   public static final String NM_LOG_DIRS = NM_PREFIX + "log-dirs";
   public static final String DEFAULT_NM_LOG_DIRS = "/tmp/logs";
 
+  /** Interval at which the delayed token removal thread runs */
+  public static final String RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
+      RM_PREFIX + "delayed.delegation-token.removal-interval-ms";
+  public static final long DEFAULT_RM_DELAYED_DELEGATION_TOKEN_REMOVAL_INTERVAL_MS =
+      30000l;
+
   /** Whether to enable log aggregation */
-  public static final String NM_LOG_AGGREGATION_ENABLED = NM_PREFIX
+  public static final String LOG_AGGREGATION_ENABLED = YARN_PREFIX
       + "log-aggregation-enable";
-  public static final boolean DEFAULT_NM_LOG_AGGREGATION_ENABLED = false;
+  public static final boolean DEFAULT_LOG_AGGREGATION_ENABLED = false;
   
   /**
    * Number of seconds to retain logs on the NodeManager. Only applicable if Log

+ 0 - 80
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/ipc/HadoopYarnRPC.java

@@ -1,80 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.ipc;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.AvroSpecificRpcEngine;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.Server;
-import org.apache.hadoop.security.token.SecretManager;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.yarn.YarnException;
-
-/**
- * This uses Hadoop RPC. Uses a tunnel AvroSpecificRpcEngine over 
- * Hadoop connection.
- * This does not give cross-language wire compatibility, since the Hadoop 
- * RPC wire format is non-standard, but it does permit use of Avro's protocol 
- * versioning features for inter-Java RPCs.
- */
-public class HadoopYarnRPC extends YarnRPC {
-
-  private static final Log LOG = LogFactory.getLog(HadoopYarnRPC.class);
-
-  @Override
-  public Object getProxy(Class protocol, InetSocketAddress addr,
-      Configuration conf) {
-    LOG.debug("Creating a HadoopYarnRpc proxy for protocol " + protocol);
-    RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
-    try {
-      return RPC.getProxy(protocol, 1, addr, conf);
-    } catch (IOException e) {
-      throw new YarnException(e);
-    }
-  }
-
-  @Override
-  public void stopProxy(Object proxy, Configuration conf) {
-    RPC.stopProxy(proxy);
-  }
-
-  @Override
-  public Server getServer(Class protocol, Object instance,
-      InetSocketAddress addr, Configuration conf,
-      SecretManager<? extends TokenIdentifier> secretManager,
-      int numHandlers) {
-    LOG.debug("Creating a HadoopYarnRpc server for protocol " + protocol + 
-        " with " + numHandlers + " handlers");
-    RPC.setProtocolEngine(conf, protocol, AvroSpecificRpcEngine.class);
-    final RPC.Server hadoopServer;
-    try {
-      hadoopServer = RPC.getServer(protocol, instance, addr.getHostName(), 
-          addr.getPort(), numHandlers, false, conf, secretManager);
-    } catch (IOException e) {
-      throw new YarnException(e);
-    }
-    return hadoopServer;
-  }
-
-}

+ 39 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/BadRequestException.java

@@ -0,0 +1,39 @@
+/**
+* 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.webapp;
+
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response.Status;
+
+public class BadRequestException extends WebApplicationException {
+
+  private static final long serialVersionUID = 1L;
+
+  public BadRequestException() {
+    super(Status.BAD_REQUEST);
+  }
+
+  public BadRequestException(java.lang.Throwable cause) {
+    super(cause, Status.BAD_REQUEST);
+  }
+
+  public BadRequestException(String msg) {
+    super(new Exception(msg), Status.BAD_REQUEST);
+  }
+}

+ 50 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/DefaultWrapperServlet.java

@@ -0,0 +1,50 @@
+/**
+* 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.webapp;
+
+import java.io.IOException;
+
+import javax.servlet.RequestDispatcher;
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletRequestWrapper;
+import javax.servlet.http.HttpServletResponse;
+
+import com.google.inject.Singleton;
+
+@Singleton
+public class DefaultWrapperServlet extends HttpServlet {
+
+private static final long serialVersionUID = 1L;
+
+public void doGet(HttpServletRequest req, HttpServletResponse resp)
+throws ServletException, IOException {
+  RequestDispatcher rd = getServletContext().getNamedDispatcher("default");
+
+  HttpServletRequest wrapped = new HttpServletRequestWrapper(req) {
+    public String getServletPath() {
+    return "";
+    }
+  };
+
+  rd.forward(wrapped, resp);
+  }
+
+}

+ 115 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java

@@ -0,0 +1,115 @@
+/**
+ * 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.webapp;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+import javax.servlet.http.HttpServletResponse;
+import javax.ws.rs.core.Context;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.authorize.AuthorizationException;
+import org.mortbay.util.ajax.JSON;
+
+import com.google.inject.Singleton;
+
+/**
+ * Handle webservices jersey exceptions and create json response in the format:
+ * { "RemoteException" :
+ *   {
+ *     "exception" : <exception type>,
+ *     "javaClassName" : <classname of exception>,
+ *     "message" : <error message from exception>
+ *   }
+ * }
+ */
+@Singleton
+@Provider
+public class GenericExceptionHandler implements ExceptionMapper<Exception> {
+  public static final Log LOG = LogFactory
+      .getLog(GenericExceptionHandler.class);
+
+  private @Context
+  HttpServletResponse response;
+
+  @Override
+  public Response toResponse(Exception e) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("GOT EXCEPITION", e);
+    }
+    // Don't catch this as filter forward on 404
+    // (ServletContainer.FEATURE_FILTER_FORWARD_ON_404)
+    // won't work and the web UI won't work!
+    if (e instanceof com.sun.jersey.api.NotFoundException) {
+      return ((com.sun.jersey.api.NotFoundException) e).getResponse();
+    }
+    // clear content type
+    response.setContentType(null);
+
+    // Convert exception
+    if (e instanceof RemoteException) {
+      e = ((RemoteException) e).unwrapRemoteException();
+    }
+
+    // Map response status
+    final Response.Status s;
+    if (e instanceof SecurityException) {
+      s = Response.Status.UNAUTHORIZED;
+    } else if (e instanceof AuthorizationException) {
+      s = Response.Status.UNAUTHORIZED;
+    } else if (e instanceof FileNotFoundException) {
+      s = Response.Status.NOT_FOUND;
+    } else if (e instanceof NotFoundException) {
+      s = Response.Status.NOT_FOUND;
+    } else if (e instanceof IOException) {
+      s = Response.Status.NOT_FOUND;
+    } else if (e instanceof UnsupportedOperationException) {
+      s = Response.Status.BAD_REQUEST;
+    } else if (e instanceof IllegalArgumentException) {
+      s = Response.Status.BAD_REQUEST;
+    } else if (e instanceof NumberFormatException) {
+      s = Response.Status.BAD_REQUEST;
+    } else if (e instanceof BadRequestException) {
+      s = Response.Status.BAD_REQUEST;
+    } else {
+      LOG.warn("INTERNAL_SERVER_ERROR", e);
+      s = Response.Status.INTERNAL_SERVER_ERROR;
+    }
+
+    // convert to json
+    final Map<String, Object> m = new TreeMap<String, Object>();
+    m.put("exception", e.getClass().getSimpleName());
+    m.put("message", e.getMessage());
+    m.put("javaClassName", e.getClass().getName());
+    final Map<String, Object> m2 = new TreeMap<String, Object>();
+    m2.put(RemoteException.class.getSimpleName(), m);
+    final String js = JSON.toString(m2);
+
+    return Response.status(s).type(MediaType.APPLICATION_JSON).entity(js)
+        .build();
+  }
+}

+ 17 - 18
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AvroSpecificRpcEngine.java → hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/NotFoundException.java

@@ -16,30 +16,29 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.ipc;
+package org.apache.hadoop.yarn.webapp;
 
-import java.io.IOException;
+import javax.ws.rs.WebApplicationException;
+import javax.ws.rs.core.Response.Status;
 
-import org.apache.avro.ipc.Responder;
-import org.apache.avro.ipc.Transceiver;
-import org.apache.avro.ipc.specific.SpecificRequestor;
-import org.apache.avro.ipc.specific.SpecificResponder;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * AvroRpcEngine which uses Avro's "specific" APIs. The protocols generated 
- * via Avro IDL needs to use this Engine.
+/*
+ * Created our own NotFoundException because com.sun.jersey.api.NotFoundException
+ * sets the Response and therefore won't be handled by the GenericExceptionhandler
+ * to fill in correct response.
  */
-@InterfaceStability.Evolving
-public class AvroSpecificRpcEngine extends AvroRpcEngine {
+public class NotFoundException extends WebApplicationException {
 
-  protected SpecificRequestor createRequestor(Class<?> protocol, 
-      Transceiver transeiver) throws IOException {
-    return new SpecificRequestor(protocol, transeiver);
+  private static final long serialVersionUID = 1L;
+
+  public NotFoundException() {
+    super(Status.NOT_FOUND);
   }
 
-  protected Responder createResponder(Class<?> iface, Object impl) {
-    return new SpecificResponder(iface, impl);
+  public NotFoundException(java.lang.Throwable cause) {
+    super(cause, Status.NOT_FOUND);
   }
 
+  public NotFoundException(String msg) {
+    super(new Exception(msg), Status.NOT_FOUND);
+  }
 }

+ 43 - 13
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApp.java

@@ -18,24 +18,29 @@
 
 package org.apache.hadoop.yarn.webapp;
 
-import com.google.common.base.CharMatcher;
-import static com.google.common.base.Preconditions.*;
-import com.google.common.base.Splitter;
-import com.google.common.collect.Lists;
-import com.google.inject.Provides;
-import com.google.inject.servlet.GuiceFilter;
-import com.google.inject.servlet.ServletModule;
+import static com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.http.HttpServer;
-import org.apache.hadoop.yarn.util.StringHelper;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.CharMatcher;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import com.google.inject.Provides;
+import com.google.inject.servlet.GuiceFilter;
+import com.google.inject.servlet.ServletModule;
+import com.sun.jersey.api.core.ResourceConfig;
+import com.sun.jersey.core.util.FeaturesAndProperties;
+import com.sun.jersey.guice.spi.container.servlet.GuiceContainer;
+import com.sun.jersey.spi.container.servlet.ServletContainer;
+
 /**
  * @see WebApps for a usage example
  */
@@ -45,9 +50,10 @@ public abstract class WebApp extends ServletModule {
   public enum HTTP { GET, POST, HEAD, PUT, DELETE };
 
   private volatile String name;
-  private volatile List<String> servePathSpecs = new ArrayList<String>(); 
+  private volatile List<String> servePathSpecs = new ArrayList<String>();
   // path to redirect to if user goes to "/"
   private volatile String redirectPath;
+  private volatile String wsName;
   private volatile Configuration conf;
   private volatile HttpServer httpServer;
   private volatile GuiceFilter guiceFilter;
@@ -104,18 +110,20 @@ public abstract class WebApp extends ServletModule {
 
   void addServePathSpec(String path) { this.servePathSpecs.add(path); }
 
-  public String[] getServePathSpecs() { 
+  public String[] getServePathSpecs() {
     return this.servePathSpecs.toArray(new String[this.servePathSpecs.size()]);
   }
 
   /**
-   * Set a path to redirect the user to if they just go to "/". For 
-   * instance "/" goes to "/yarn/apps". This allows the filters to 
+   * Set a path to redirect the user to if they just go to "/". For
+   * instance "/" goes to "/yarn/apps". This allows the filters to
    * more easily differentiate the different webapps.
    * @param path  the path to redirect to
    */
   void setRedirectPath(String path) { this.redirectPath = path; }
 
+  void setWebServices (String name) { this.wsName = name; }
+
   public String getRedirectPath() { return this.redirectPath; }
 
   void setHostClass(Class<?> cls) {
@@ -129,10 +137,32 @@ public abstract class WebApp extends ServletModule {
   @Override
   public void configureServlets() {
     setup();
+
     serve("/", "/__stop").with(Dispatcher.class);
+
     for (String path : this.servePathSpecs) {
       serve(path).with(Dispatcher.class);
     }
+
+    // Add in the web services filters/serves if app has them.
+    // Using Jersey/guice integration module. If user has web services
+    // they must have also bound a default one in their webapp code.
+    if (this.wsName != null) {
+      // There seems to be an issue with the guice/jersey integration
+      // where we have to list the stuff we don't want it to serve
+      // through the guicecontainer. In this case its everything except
+      // the the web services api prefix. We can't just change the filter
+      // from /* below - that doesn't work.
+      String regex = "(?!/" + this.wsName + ")";
+      serveRegex(regex).with(DefaultWrapperServlet.class);
+
+      Map<String, String> params = new HashMap<String, String>();
+      params.put(ResourceConfig.FEATURE_IMPLICIT_VIEWABLES, "true");
+      params.put(ServletContainer.FEATURE_FILTER_FORWARD_ON_404, "true");
+      params.put(FeaturesAndProperties.FEATURE_XMLROOTELEMENT_PROCESSING, "true");
+      filter("/*").through(GuiceContainer.class, params);
+    }
+
   }
 
   /**

+ 30 - 1
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java

@@ -72,6 +72,7 @@ public class WebApps {
     }
     
     final String name;
+    final String wsName;
     final Class<T> api;
     final T application;
     String bindAddress = "0.0.0.0";
@@ -82,10 +83,15 @@ public class WebApps {
     private final HashSet<ServletStruct> servlets = new HashSet<ServletStruct>();
     private final HashMap<String, Object> attributes = new HashMap<String, Object>();
 
-    Builder(String name, Class<T> api, T application) {
+    Builder(String name, Class<T> api, T application, String wsName) {
       this.name = name;
       this.api = api;
       this.application = application;
+      this.wsName = wsName;
+    }
+
+    Builder(String name, Class<T> api, T application) {
+      this(name, api, application, null);
     }
 
     public Builder<T> at(String bindAddress) {
@@ -142,6 +148,7 @@ public class WebApps {
         };
       }
       webapp.setName(name);
+      webapp.setWebServices(wsName);
       String basePath = "/" + name;
       webapp.setRedirectPath(basePath);
       if (basePath.equals("/")) { 
@@ -150,6 +157,14 @@ public class WebApps {
         webapp.addServePathSpec(basePath);
         webapp.addServePathSpec(basePath + "/*");
       }
+      if (wsName != null && !wsName.equals(basePath)) {
+        if (wsName.equals("/")) { 
+          webapp.addServePathSpec("/*");
+        } else {
+          webapp.addServePathSpec("/" + wsName);
+          webapp.addServePathSpec("/" + wsName + "/*");
+        }
+      }
       if (conf == null) {
         conf = new Configuration();
       }
@@ -231,6 +246,20 @@ public class WebApps {
     }
   }
 
+  /**
+   * Create a new webapp builder.
+   * @see WebApps for a complete example
+   * @param <T> application (holding the embedded webapp) type
+   * @param prefix of the webapp
+   * @param api the api class for the application
+   * @param app the application instance
+   * @param wsPrefix the prefix for the webservice api for this app
+   * @return a webapp builder
+   */
+  public static <T> Builder<T> $for(String prefix, Class<T> api, T app, String wsPrefix) {
+    return new Builder<T>(prefix, api, app, wsPrefix);
+  }
+
   /**
    * Create a new webapp builder.
    * @see WebApps for a complete example

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/log/AggregatedLogsBlock.java

@@ -53,8 +53,8 @@ public class AggregatedLogsBlock extends HtmlBlock {
       logEntity = containerId.toString();
     }
 
-    if (!conf.getBoolean(YarnConfiguration.NM_LOG_AGGREGATION_ENABLED,
-        YarnConfiguration.DEFAULT_NM_LOG_AGGREGATION_ENABLED)) {
+    if (!conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
+        YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)) {
       html.h1()
           ._("Aggregation is not enabled. Try the nodemanager at " + nodeId)
           ._();

+ 0 - 10
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -58,16 +58,6 @@ public class TestRPC {
   private static final String EXCEPTION_CAUSE = "exception cause";
   private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
   
-//  @Test
-//  public void testAvroRPC() throws Exception {
-//    test(AvroYarnRPC.class.getName());
-//  }
-//
-//  @Test
-//  public void testHadoopNativeRPC() throws Exception {
-//    test(HadoopYarnRPC.class.getName());
-//  }
-
   @Test
   public void testUnknownCall() {
     Configuration conf = new Configuration();

+ 0 - 40
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/avro/ResourceTracker.genavro

@@ -1,40 +0,0 @@
-@namespace("org.apache.hadoop.yarn")
-protocol ResourceTracker {
-
-  import idl "yarn/yarn-api/src/main/avro/yarn-types.genavro";
-  
-  // ResourceTracker
-  record NodeID {
-    int id;
-  }
-
-  record NodeHealthStatus {
-    boolean isNodeHealthy;
-    union {string, null} healthReport;
-    long lastHealthReportTime;
-  }
-
-  record NodeStatus {
-    NodeID nodeId;
-	int responseId;
-	long lastSeen;
-    map<array<org.apache.hadoop.yarn.Container>> containers;
-	NodeHealthStatus nodeHealthStatus;
-  }
-
-  record RegistrationResponse {
-    NodeID nodeID;
-    union {bytes, null} secretKey;
-  }
-
-  record HeartbeatResponse {
-    int responseId;
-    boolean reboot;
-    array<org.apache.hadoop.yarn.Container> containersToCleanup;
-    array<org.apache.hadoop.yarn.ApplicationID> appplicationsToCleanup;
-  }
-  
-  RegistrationResponse registerNodeManager(string node, org.apache.hadoop.yarn.Resource resource) throws YarnRemoteException;
-  HeartbeatResponse nodeHeartbeat(NodeStatus nodeStatus) throws YarnRemoteException;
-  
-}

+ 4 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/NodeStatus.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.api.records;
 
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
@@ -33,6 +34,9 @@ public interface NodeStatus {
   public abstract void setContainersStatuses(
       List<ContainerStatus> containersStatuses);
 
+  public abstract List<ApplicationId> getKeepAliveApplications();
+  public abstract void setKeepAliveApplications(List<ApplicationId> appIds);
+  
   NodeHealthStatus getNodeHealthStatus();
   void setNodeHealthStatus(NodeHealthStatus healthStatus);
 

+ 98 - 18
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/NodeStatusPBImpl.java

@@ -23,13 +23,16 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeHealthStatusPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeHealthStatusProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
@@ -37,7 +40,9 @@ import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerCommonProtos.NodeStatusProtoOrBuilder;
 import org.apache.hadoop.yarn.server.api.records.NodeStatus;
     
-public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements NodeStatus {
+
+public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements
+    NodeStatus {
   NodeStatusProto proto = NodeStatusProto.getDefaultInstance();
   NodeStatusProto.Builder builder = null;
   boolean viaProto = false;
@@ -45,6 +50,7 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
   private NodeId nodeId = null;
   private List<ContainerStatus> containers = null;
   private NodeHealthStatus nodeHealthStatus = null;
+  private List<ApplicationId> keepAliveApplications = null;
   
   public NodeStatusPBImpl() {
     builder = NodeStatusProto.newBuilder();
@@ -55,15 +61,14 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     viaProto = true;
   }
   
-  public NodeStatusProto getProto() {
-
-      mergeLocalToProto();
+  public synchronized NodeStatusProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
   }
 
-  private void mergeLocalToBuilder() {
+  private synchronized void mergeLocalToBuilder() {
     if (this.nodeId != null) {
       builder.setNodeId(convertToProtoFormat(this.nodeId));
     }
@@ -73,9 +78,12 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     if (this.nodeHealthStatus != null) {
       builder.setNodeHealthStatus(convertToProtoFormat(this.nodeHealthStatus));
     }
+    if (this.keepAliveApplications != null) {
+      addKeepAliveApplicationsToProto();
+    }
   }
 
-  private void mergeLocalToProto() {
+  private synchronized void mergeLocalToProto() {
     if (viaProto) 
       maybeInitBuilder();
     mergeLocalToBuilder();
@@ -84,14 +92,14 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     viaProto = true;
   }
 
-  private void maybeInitBuilder() {
+  private synchronized void maybeInitBuilder() {
     if (viaProto || builder == null) {
       builder = NodeStatusProto.newBuilder(proto);
     }
     viaProto = false;
   }
     
-  private void addContainersToProto() {
+  private synchronized void addContainersToProto() {
     maybeInitBuilder();
     builder.clearContainersStatuses();
     if (containers == null)
@@ -124,19 +132,53 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     };
     builder.addAllContainersStatuses(iterable);
   }
+  
+  private synchronized void addKeepAliveApplicationsToProto() {
+    maybeInitBuilder();
+    builder.clearKeepAliveApplications();
+    if (keepAliveApplications == null)
+      return;
+    Iterable<ApplicationIdProto> iterable = new Iterable<ApplicationIdProto>() {
+      @Override
+      public Iterator<ApplicationIdProto> iterator() {
+        return new Iterator<ApplicationIdProto>() {
+  
+          Iterator<ApplicationId> iter = keepAliveApplications.iterator();
+  
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+  
+          @Override
+          public ApplicationIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+  
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+  
+          }
+        };
+  
+      }
+    };
+    builder.addAllKeepAliveApplications(iterable);
+  }
 
   @Override
-  public int getResponseId() {
+  public synchronized int getResponseId() {
     NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
     return p.getResponseId();
   }
   @Override
-  public void setResponseId(int responseId) {
+  public synchronized void setResponseId(int responseId) {
     maybeInitBuilder();
     builder.setResponseId(responseId);
   }
   @Override
-  public NodeId getNodeId() {
+  public synchronized NodeId getNodeId() {
     NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
     if (this.nodeId != null) {
       return this.nodeId;
@@ -148,8 +190,9 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     
     return this.nodeId;
   }
+  
   @Override
-  public void setNodeId(NodeId nodeId) {
+  public synchronized void setNodeId(NodeId nodeId) {
     maybeInitBuilder();
     if (nodeId == null)
       builder.clearNodeId();
@@ -158,20 +201,35 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
   }
   
   @Override
-  public List<ContainerStatus> getContainersStatuses() {
+  public synchronized List<ContainerStatus> getContainersStatuses() {
     initContainers();
     return this.containers;
   }
 
   @Override
-  public void setContainersStatuses(List<ContainerStatus> containers) {
+  public synchronized void setContainersStatuses(
+      List<ContainerStatus> containers) {
     if (containers == null) {
       builder.clearContainersStatuses();
     }
     this.containers = containers;
   }
+  
+  @Override
+  public synchronized List<ApplicationId> getKeepAliveApplications() {
+    initKeepAliveApplications();
+    return this.keepAliveApplications;
+  }
+  
+  @Override
+  public synchronized void setKeepAliveApplications(List<ApplicationId> appIds) {
+    if (appIds == null) {
+      builder.clearKeepAliveApplications();
+    }
+    this.keepAliveApplications = appIds;
+  }
 
-  private void initContainers() {
+  private synchronized void initContainers() {
     if (this.containers != null) {
       return;
     }
@@ -185,8 +243,22 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
     
   }
   
+  private synchronized void initKeepAliveApplications() {
+    if (this.keepAliveApplications != null) {
+      return;
+    }
+    NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
+    List<ApplicationIdProto> list = p.getKeepAliveApplicationsList();
+    this.keepAliveApplications = new ArrayList<ApplicationId>();
+
+    for (ApplicationIdProto c : list) {
+      this.keepAliveApplications.add(convertFromProtoFormat(c));
+    }
+    
+  }
+  
   @Override
-  public NodeHealthStatus getNodeHealthStatus() {
+  public synchronized NodeHealthStatus getNodeHealthStatus() {
     NodeStatusProtoOrBuilder p = viaProto ? proto : builder;
     if (nodeHealthStatus != null) {
       return nodeHealthStatus;
@@ -199,7 +271,7 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
   }
 
   @Override
-  public void setNodeHealthStatus(NodeHealthStatus healthStatus) {
+  public synchronized void setNodeHealthStatus(NodeHealthStatus healthStatus) {
     maybeInitBuilder();
     if (healthStatus == null) {
       builder.clearNodeHealthStatus();
@@ -231,4 +303,12 @@ public class NodeStatusPBImpl extends ProtoBase<NodeStatusProto> implements Node
   private ContainerStatusProto convertToProtoFormat(ContainerStatus c) {
     return ((ContainerStatusPBImpl)c).getProto();
   }
-}  
+  
+  private ApplicationIdPBImpl convertFromProtoFormat(ApplicationIdProto c) {
+    return new ApplicationIdPBImpl(c);
+  }
+  
+  private ApplicationIdProto convertToProtoFormat(ApplicationId c) {
+    return ((ApplicationIdPBImpl)c).getProto();
+  }
+}

+ 1 - 0
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/proto/yarn_server_common_protos.proto

@@ -34,6 +34,7 @@ message NodeStatusProto {
   optional int32 response_id = 2;
   repeated ContainerStatusProto containersStatuses = 3;
   optional NodeHealthStatusProto nodeHealthStatus = 4;
+  repeated ApplicationIdProto keep_alive_applications = 5;
 }
 
 message RegistrationResponseProto {

Some files were not shown because too many files changed in this diff