瀏覽代碼

YARN-954. Implemented web UI for the ApplicationHistoryServer and wired it into the HistoryStorage. Contributed by Zhijie Shen.
svn merge --ignore-ancestry -c 1556745 ../YARN-321


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1562200 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 11 年之前
父節點
當前提交
20664c0460
共有 27 個文件被更改,包括 1907 次插入133 次删除
  1. 3 0
      hadoop-yarn-project/CHANGES.txt
  2. 25 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
  3. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
  4. 10 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java
  5. 18 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
  6. 0 29
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/AHSWebServer.java
  7. 2 97
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
  8. 11 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java
  9. 39 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
  10. 50 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSController.java
  11. 95 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
  12. 49 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
  13. 39 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
  14. 72 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
  15. 73 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
  16. 41 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java
  17. 58 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/JAXBContextResolver.java
  18. 51 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java
  19. 180 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java
  20. 121 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java
  21. 160 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
  22. 188 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
  23. 150 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
  24. 100 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
  25. 84 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java
  26. 169 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
  27. 118 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java

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

@@ -505,6 +505,9 @@ Branch YARN-321: Generic ApplicationHistoryService
   YARN-1242. Changed yarn scripts to be able to start ApplicationHistoryServer
   YARN-1242. Changed yarn scripts to be able to start ApplicationHistoryServer
   as an individual process. (Mayank Bansal via vinodkv)
   as an individual process. (Mayank Bansal via vinodkv)
 
 
+  YARN-954. Implemented web UI for the ApplicationHistoryServer and wired it into
+  the HistoryStorage. (Zhijie Shen via vinodkv)
+
 Release 2.2.0 - 2013-10-13
 Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 25 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java

@@ -974,6 +974,31 @@ public class YarnConfiguration extends Configuration {
       + "client.thread-count";
       + "client.thread-count";
   public static final int DEFAULT_AHS_CLIENT_THREAD_COUNT = 10;
   public static final int DEFAULT_AHS_CLIENT_THREAD_COUNT = 10;
   
   
+
+  /** The address of the AHS web application.*/
+  public static final String AHS_WEBAPP_ADDRESS = AHS_PREFIX
+      + "webapp.address";
+
+  public static final int DEFAULT_AHS_WEBAPP_PORT = 8188;
+  public static final String DEFAULT_AHS_WEBAPP_ADDRESS = "0.0.0.0:"
+      + DEFAULT_AHS_WEBAPP_PORT;
+
+  /** The https address of the AHS web application.*/
+  public static final String AHS_WEBAPP_HTTPS_ADDRESS = AHS_PREFIX
+      + "webapp.https.address";
+
+  public static final int DEFAULT_AHS_WEBAPP_HTTPS_PORT = 8190;
+  public static final String DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS = "0.0.0.0:"
+      + DEFAULT_AHS_WEBAPP_HTTPS_PORT;
+
+  /**The kerberos principal to be used for spnego filter for AHS.*/
+  public static final String AHS_WEBAPP_SPNEGO_USER_NAME_KEY =
+      AHS_PREFIX + "webapp.spnego-principal";
+
+  /**The kerberos keytab to be used for spnego filter for AHS.*/
+  public static final String AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
+      AHS_PREFIX + "webapp.spnego-keytab-file";
+
   ////////////////////////////////
   ////////////////////////////////
   // Other Configs
   // Other Configs
   ////////////////////////////////
   ////////////////////////////////

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java

@@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 public interface YarnWebParams {
 public interface YarnWebParams {
   String NM_NODENAME = "nm.id";
   String NM_NODENAME = "nm.id";
   String APPLICATION_ID = "app.id";
   String APPLICATION_ID = "app.id";
+  String APPLICATION_ATTEMPT_ID = "appattempt.id";
   String CONTAINER_ID = "container.id";
   String CONTAINER_ID = "container.id";
   String CONTAINER_LOG_TYPE= "log.type";
   String CONTAINER_LOG_TYPE= "log.type";
   String ENTITY_STRING = "entity.string";
   String ENTITY_STRING = "entity.string";

+ 10 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/util/WebAppUtils.java

@@ -144,6 +144,16 @@ public class WebAppUtils {
         YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS);
         YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS);
     }
     }
   }
   }
+
+  public static String getAHSWebAppURLWithoutScheme(Configuration conf) {
+    if (HttpConfig.isSecure()) {
+      return conf.get(YarnConfiguration.AHS_WEBAPP_HTTPS_ADDRESS,
+        YarnConfiguration.DEFAULT_AHS_WEBAPP_HTTPS_ADDRESS);
+    } else {
+      return conf.get(YarnConfiguration.AHS_WEBAPP_ADDRESS,
+        YarnConfiguration.DEFAULT_AHS_WEBAPP_ADDRESS);
+    }
+  }
   
   
   /**
   /**
    * if url has scheme then it will be returned as it is else it will return
    * if url has scheme then it will be returned as it is else it will return

+ 18 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml

@@ -1064,6 +1064,24 @@
 
 
   <!-- Application History Service's Configuration-->
   <!-- Application History Service's Configuration-->
 
 
+  <property>
+    <description>The hostname of the AHS.</description>
+    <name>yarn.ahs.hostname</name>
+    <value>0.0.0.0</value>
+  </property>
+
+  <property>
+    <description>The http address of the AHS web application.</description>
+    <name>yarn.ahs.webapp.address</name>
+    <value>${yarn.ahs.hostname}:8188</value>
+  </property>
+
+  <property>
+    <description>The https adddress of the AHS web application.</description>
+    <name>yarn.ahs.webapp.https.address</name>
+    <value>${yarn.ahs.hostname}:8190</value>
+  </property>
+
   <property>
   <property>
     <description>URI pointing to the location of the FileSystem path where
     <description>URI pointing to the location of the FileSystem path where
     the history will be persisted. This must be supplied when using
     the history will be persisted. This must be supplied when using

+ 0 - 29
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/AHSWebServer.java

@@ -1,29 +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.server.applicationhistoryservice;
-
-import org.apache.hadoop.service.AbstractService;
-
-public class AHSWebServer extends AbstractService {
-
-  public AHSWebServer() {
-    super(AHSWebServer.class.getName());
-  }
-
-}

+ 2 - 97
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java

@@ -18,106 +18,11 @@
 
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice;
 package org.apache.hadoop.yarn.server.applicationhistoryservice;
 
 
-import java.io.IOException;
-import java.util.Map;
-
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
-import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
 
 
 @InterfaceAudience.Public
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 @InterfaceStability.Unstable
-public interface ApplicationHistoryManager {
-  /**
-   * This method returns Application {@link ApplicationReport} for the specified
-   * {@link ApplicationId}.
-   * 
-   * @return {@link ApplicationReport} for the ApplicationId.
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  ApplicationReport getApplication(ApplicationId appId) throws IOException;
-
-  /**
-   * This method returns all Application {@link ApplicationReport}s
-   * 
-   * @return map {@link ApplicationId, @link ApplicationReport}s.
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  Map<ApplicationId, ApplicationReport> getAllApplications() throws IOException;
-
-  /**
-   * Application can have multiple application attempts
-   * {@link ApplicationAttemptReport}. This method returns the all
-   * {@link ApplicationAttemptReport}s for the Application.
-   * 
-   * @return all {@link ApplicationAttemptReport}s for the Application.
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  Map<ApplicationAttemptId, ApplicationAttemptReport> getApplicationAttempts(
-      ApplicationId appId) throws IOException;
-
-  /**
-   * This method returns {@link ApplicationAttemptReport} for specified
-   * {@link ApplicationId}.
-   * 
-   * @param {@link ApplicationAttemptId}
-   * @return {@link ApplicationAttemptReport} for ApplicationAttemptId
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  ApplicationAttemptReport getApplicationAttempt(
-      ApplicationAttemptId appAttemptId) throws IOException;
-
-  /**
-   * This method returns {@link ContainerReport} for specified
-   * {@link ContainerId}.
-   * 
-   * @param {@link ContainerId}
-   * @return {@link Container} for ContainerId
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  ContainerReport getContainer(ContainerId containerId) throws IOException;
-
-  /**
-   * This method returns {@link ContainerReport} for specified
-   * {@link ApplicationAttemptId}.
-   * 
-   * @param {@link ApplicationAttemptId}
-   * @return {@link Container} for ApplicationAttemptId
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
-      throws IOException;
-
-  /**
-   * This method returns Map{@link ContainerId,@link ContainerReport} for
-   * specified {@link ApplicationAttemptId}.
-   * 
-   * @param {@link ApplicationAttemptId}
-   * @return Map{@link ContainerId, @link ContainerReport} for
-   *         ApplicationAttemptId
-   * @throws {@link IOException}
-   */
-  @Public
-  @Unstable
-  Map<ContainerId, ContainerReport> getContainers(
-      ApplicationAttemptId appAttemptId) throws IOException;
+public interface ApplicationHistoryManager extends ApplicationContext {
 }
 }

+ 11 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerImpl.java

@@ -56,9 +56,7 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
   @Override
   @Override
   protected void serviceInit(Configuration conf) throws Exception {
   protected void serviceInit(Configuration conf) throws Exception {
     LOG.info("ApplicationHistory Init");
     LOG.info("ApplicationHistory Init");
-    historyStore = ReflectionUtils.newInstance(conf.getClass(
-        YarnConfiguration.AHS_STORE, FileSystemApplicationHistoryStore.class,
-        ApplicationHistoryStore.class), conf);
+    historyStore = createApplicationHistoryStore(conf);
     historyStore.init(conf);
     historyStore.init(conf);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
@@ -77,6 +75,13 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
     super.serviceStop();
     super.serviceStop();
   }
   }
 
 
+  protected ApplicationHistoryStore createApplicationHistoryStore(
+      Configuration conf) {
+    return ReflectionUtils.newInstance(conf.getClass(
+        YarnConfiguration.AHS_STORE, FileSystemApplicationHistoryStore.class,
+        ApplicationHistoryStore.class), conf);
+  }
+
   @Override
   @Override
   public ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
   public ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
       throws IOException {
       throws IOException {
@@ -148,8 +153,9 @@ public class ApplicationHistoryManagerImpl extends AbstractService implements
     return ApplicationAttemptReport.newInstance(appAttemptHistory
     return ApplicationAttemptReport.newInstance(appAttemptHistory
         .getApplicationAttemptId(), appAttemptHistory.getHost(),
         .getApplicationAttemptId(), appAttemptHistory.getHost(),
         appAttemptHistory.getRPCPort(), appAttemptHistory.getTrackingURL(),
         appAttemptHistory.getRPCPort(), appAttemptHistory.getTrackingURL(),
-        appAttemptHistory.getDiagnosticsInfo(), null, appAttemptHistory
-            .getMasterContainerId());
+        appAttemptHistory.getDiagnosticsInfo(),
+        appAttemptHistory.getYarnApplicationAttemptState(),
+        appAttemptHistory.getMasterContainerId());
   }
   }
 
 
   @Override
   @Override

+ 39 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java

@@ -31,6 +31,11 @@ import org.apache.hadoop.util.ShutdownHookManager;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.WebApps;
+import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
 
 
@@ -46,6 +51,7 @@ public class ApplicationHistoryServer extends CompositeService {
 
 
   ApplicationHistoryClientService ahsClientService;
   ApplicationHistoryClientService ahsClientService;
   ApplicationHistoryManager historyManager;
   ApplicationHistoryManager historyManager;
+  private WebApp webApp;
 
 
   public ApplicationHistoryServer() {
   public ApplicationHistoryServer() {
     super(ApplicationHistoryServer.class.getName());
     super(ApplicationHistoryServer.class.getName());
@@ -57,8 +63,6 @@ public class ApplicationHistoryServer extends CompositeService {
     ahsClientService = createApplicationHistoryClientService(historyManager);
     ahsClientService = createApplicationHistoryClientService(historyManager);
     addService(ahsClientService);
     addService(ahsClientService);
     addService((Service) historyManager);
     addService((Service) historyManager);
-    AHSWebServer webServer = new AHSWebServer();
-    addService(webServer);
     super.serviceInit(conf);
     super.serviceInit(conf);
   }
   }
 
 
@@ -66,11 +70,17 @@ public class ApplicationHistoryServer extends CompositeService {
   protected void serviceStart() throws Exception {
   protected void serviceStart() throws Exception {
     DefaultMetricsSystem.initialize("ApplicationHistoryServer");
     DefaultMetricsSystem.initialize("ApplicationHistoryServer");
     JvmMetrics.initSingleton("ApplicationHistoryServer", null);
     JvmMetrics.initSingleton("ApplicationHistoryServer", null);
+
+    startWebApp();
     super.serviceStart();
     super.serviceStart();
   }
   }
 
 
   @Override
   @Override
   protected void serviceStop() throws Exception {
   protected void serviceStop() throws Exception {
+    if (webApp != null) {
+      webApp.stop();
+    }
+
     DefaultMetricsSystem.shutdown();
     DefaultMetricsSystem.shutdown();
     super.serviceStop();
     super.serviceStop();
   }
   }
@@ -118,4 +128,31 @@ public class ApplicationHistoryServer extends CompositeService {
   public static void main(String[] args) {
   public static void main(String[] args) {
     launchAppHistoryServer(args);
     launchAppHistoryServer(args);
   }
   }
+
+  protected ApplicationHistoryManager createApplicationHistoryManager(
+      Configuration conf) {
+    return new ApplicationHistoryManagerImpl();
+  }
+
+  protected void startWebApp() {
+    String bindAddress = WebAppUtils.getAHSWebAppURLWithoutScheme(getConfig());
+    LOG.info("Instantiating AHSWebApp at " + bindAddress);
+    try {
+      webApp = WebApps
+          .$for("applicationhistory", ApplicationHistoryClientService.class,
+              ahsClientService, "ws")
+          .with(getConfig())
+          .withHttpSpnegoPrincipalKey(
+              YarnConfiguration.AHS_WEBAPP_SPNEGO_USER_NAME_KEY)
+          .withHttpSpnegoKeytabKey(
+              YarnConfiguration.AHS_WEBAPP_SPNEGO_KEYTAB_FILE_KEY)
+          .at(bindAddress)
+          .start(new AHSWebApp(historyManager));
+    } catch (Exception e) {
+      String msg = "AHSWebApp failed to start.";
+      LOG.error(msg, e);
+      throw new YarnRuntimeException(msg, e);
+    }
+  }
+
 }
 }

+ 50 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSController.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.server.applicationhistoryservice.webapp;
+
+import org.apache.hadoop.yarn.webapp.Controller;
+
+import com.google.inject.Inject;
+
+
+public class AHSController extends Controller {
+
+  @Inject
+  AHSController(RequestContext ctx) {
+    super(ctx);
+  }
+
+  @Override
+  public void index() {
+    setTitle("Application History");
+  }
+
+  public void app() {
+    render(AppPage.class);
+  }
+
+  public void appattempt() {
+    render(AppAttemptPage.class);
+  }
+
+  public void container() {
+    render(ContainerPage.class);
+  }
+
+}

+ 95 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java

@@ -0,0 +1,95 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.sjoin;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.ACCORDION_ID;
+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 org.apache.hadoop.yarn.server.webapp.AppsBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
+
+// Do NOT rename/refactor this to AHSView as it will wreak havoc
+// on Mac OS HFS
+public class AHSView extends TwoColumnLayout {
+  static final int MAX_DISPLAY_ROWS = 100;  // direct table rendering
+  static final int MAX_FAST_ROWS = 1000;    // inline js array
+
+  @Override
+  protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+    set(DATATABLES_ID, "apps");
+    set(initID(DATATABLES, "apps"), appsTableInit());
+    setTableStyles(html, "apps", ".queue {width:6em}", ".ui {width:8em}");
+
+    // Set the correct title.
+    String reqState = $(APP_STATE);
+    reqState = (reqState == null || reqState.isEmpty() ? "All" : reqState);
+    setTitle(sjoin(reqState, "Applications"));
+  }
+
+  protected void commonPreHead(Page.HTML<_> html) {
+    set(ACCORDION_ID, "nav");
+    set(initID(ACCORDION, "nav"), "{autoHeight:false, active:0}");
+  }
+
+  @Override
+  protected Class<? extends SubView> nav() {
+    return NavBlock.class;
+  }
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return AppsBlock.class;
+  }
+
+  private String appsTableInit() {
+    // id, user, name, queue, starttime, finishtime, state, status, progress, ui
+    return tableInit()
+      .append(", 'aaData': appsTableData")
+      .append(", bDeferRender: true")
+      .append(", bProcessing: true")
+
+      .append("\n, aoColumnDefs: ")
+      .append(getAppsTableColumnDefs())
+
+      // Sort by id upon page load
+      .append(", aaSorting: [[0, 'desc']]}").toString();
+  }
+
+  protected String getAppsTableColumnDefs() {
+    StringBuilder sb = new StringBuilder();
+    return sb
+      .append("[\n")
+      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append(", 'mRender': parseHadoopID }")
+
+      .append("\n, {'sType':'numeric', 'aTargets': [5, 6]")
+      .append(", 'mRender': renderHadoopDate }")
+
+      .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [9]")
+      .append(", 'mRender': parseHadoopProgress }]").toString();
+  }
+}

+ 49 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.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.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
+
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
+import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
+import org.apache.hadoop.yarn.webapp.WebApp;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+public class AHSWebApp extends WebApp implements YarnWebParams {
+
+  private final ApplicationHistoryManager applicationHistoryManager;
+
+  public AHSWebApp(ApplicationHistoryManager applicationHistoryManager) {
+    this.applicationHistoryManager = applicationHistoryManager;
+  }
+
+  @Override
+  public void setup() {
+    bind(JAXBContextResolver.class);
+    bind(AHSWebServices.class);
+    bind(GenericExceptionHandler.class);
+    bind(ApplicationContext.class).toInstance(applicationHistoryManager);
+    route("/", AHSController.class);
+    route(pajoin("/apps", APP_STATE), AHSController.class);
+    route(pajoin("/app", APPLICATION_ID), AHSController.class, "app");
+    route(pajoin("/appattempt", APPLICATION_ATTEMPT_ID), AHSController.class,
+        "appattempt");
+    route(pajoin("/container", CONTAINER_ID), AHSController.class, "container");
+  }
+}

+ 39 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.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.server.applicationhistoryservice.webapp;
+
+import javax.ws.rs.Path;
+
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+
+@Singleton
+@Path("/ws/v1/applicationhistory")
+public class AHSWebServices {
+
+  private ApplicationContext appContext;
+
+  @Inject
+  public AHSWebServices(ApplicationContext appContext) {
+    this.appContext = appContext;
+  }
+
+}

+ 72 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java

@@ -0,0 +1,72 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+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 org.apache.hadoop.yarn.server.webapp.AppAttemptBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+public class AppAttemptPage extends AHSView {
+
+  @Override
+  protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+
+    String appAttemptId = $(YarnWebParams.APPLICATION_ATTEMPT_ID);
+    set(TITLE, appAttemptId.isEmpty() ?
+        "Bad request: missing application attempt ID" : join(
+        "Application Attempt ", $(YarnWebParams.APPLICATION_ATTEMPT_ID)));
+
+    set(DATATABLES_ID, "containers");
+    set(initID(DATATABLES, "containers"), containersTableInit());
+    setTableStyles(html, "containers", ".queue {width:6em}", ".ui {width:8em}");
+  }
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return AppAttemptBlock.class;
+  }
+
+  private String containersTableInit() {
+    return tableInit()
+      .append(", 'aaData': containersTableData")
+      .append(", bDeferRender: true")
+      .append(", bProcessing: true")
+
+      .append("\n, aoColumnDefs: ")
+      .append(getContainersTableColumnDefs())
+
+      // Sort by id upon page load
+      .append(", aaSorting: [[0, 'desc']]}").toString();
+  }
+
+  protected String getContainersTableColumnDefs() {
+    StringBuilder sb = new StringBuilder();
+    return sb
+      .append("[\n")
+      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append(", 'mRender': parseHadoopID }]").toString();
+  }
+
+}

+ 73 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java

@@ -0,0 +1,73 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+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 org.apache.hadoop.yarn.server.webapp.AppBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+public class AppPage extends AHSView {
+
+  @Override protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+
+    String appId = $(YarnWebParams.APPLICATION_ID);
+    set(TITLE, appId.isEmpty() ?
+        "Bad request: missing application ID" : join(
+        "Application ", $(YarnWebParams.APPLICATION_ID)));
+    
+    set(DATATABLES_ID, "attempts");
+    set(initID(DATATABLES, "attempts"), attemptsTableInit());
+    setTableStyles(html, "attempts", ".queue {width:6em}", ".ui {width:8em}");
+  }
+
+  @Override protected Class<? extends SubView> content() {
+    return AppBlock.class;
+  }
+
+  private String attemptsTableInit() {
+    return tableInit()
+      .append(", 'aaData': attemptsTableData")
+      .append(", bDeferRender: true")
+      .append(", bProcessing: true")
+
+      .append("\n, aoColumnDefs: ")
+      .append(getAttemptsTableColumnDefs())
+
+      // Sort by id upon page load
+      .append(", aaSorting: [[0, 'desc']]}").toString();
+  }
+
+  protected String getAttemptsTableColumnDefs() {
+    StringBuilder sb = new StringBuilder();
+    return sb
+      .append("[\n")
+      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append(", 'mRender': parseHadoopID }")
+
+      .append("\n, {'sType':'numeric', 'aTargets': [1]")
+      .append(", 'mRender': renderHadoopDate }]").toString();
+  }
+}

+ 41 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/ContainerPage.java

@@ -0,0 +1,41 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+
+import org.apache.hadoop.yarn.server.webapp.ContainerBlock;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+public class ContainerPage extends AHSView {
+
+  @Override
+  protected void preHead(Page.HTML<_> html) {
+    commonPreHead(html);
+
+    String containerId = $(YarnWebParams.CONTAINER_ID);
+    set(TITLE, containerId.isEmpty() ?
+        "Bad request: missing container ID" : join(
+        "Container ", $(YarnWebParams.CONTAINER_ID)));
+  }
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return ContainerBlock.class;
+  }}

+ 58 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/JAXBContextResolver.java

@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+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.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+
+import com.google.inject.Singleton;
+import com.sun.jersey.api.json.JSONConfiguration;
+import com.sun.jersey.api.json.JSONJAXBContext;
+
+@Singleton
+@Provider
+@SuppressWarnings("rawtypes")
+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, AppAttemptInfo.class, ContainerInfo.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;
+  }
+}

+ 51 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java

@@ -0,0 +1,51 @@
+/**
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+
+public class NavBlock extends HtmlBlock {
+
+  @Override
+  public void render(Block html) {
+    html.
+        div("#nav").
+            h3("Application History").
+                ul().
+                    li().a(url("apps"), "Applications").
+                        ul().
+                            li().a(url("apps",
+                                YarnApplicationState.FINISHED.toString()),
+                                YarnApplicationState.FINISHED.toString()).
+                            _().
+                            li().a(url("apps",
+                                YarnApplicationState.FAILED.toString()),
+                                YarnApplicationState.FAILED.toString()).
+                            _().
+                            li().a(url("apps",
+                                YarnApplicationState.KILLED.toString()),
+                                YarnApplicationState.KILLED.toString()).
+                            _().
+                        _().
+                    _().
+                _().
+            _();
+  }
+}

+ 180 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java

@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
+
+import static org.apache.hadoop.yarn.webapp.Params.TITLE;
+import static org.mockito.Mockito.mock;
+import junit.framework.Assert;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerImpl;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStoreTestUtils;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.MemoryApplicationHistoryStore;
+import org.apache.hadoop.yarn.util.StringHelper;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+import org.apache.hadoop.yarn.webapp.test.WebAppTests;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.inject.Injector;
+
+public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
+
+  @Before
+  public void setup() {
+    store = new MemoryApplicationHistoryStore();
+  }
+
+  @Test
+  public void testAppControllerIndex() throws Exception {
+    ApplicationHistoryManager ahManager = mock(ApplicationHistoryManager.class);
+    Injector injector =
+        WebAppTests.createMockInjector(ApplicationHistoryManager.class,
+            ahManager);
+    AHSController controller =
+        injector.getInstance(AHSController.class);
+    controller.index();
+    Assert
+        .assertEquals("Application History", controller.get(TITLE, "unknown"));
+  }
+
+  @Test
+  public void testView() throws Exception {
+    Injector injector =
+        WebAppTests.createMockInjector(ApplicationContext.class,
+            mockApplicationHistoryManager(5, 1, 1));
+    AHSView ahsViewInstance = injector.getInstance(AHSView.class);
+
+    ahsViewInstance.render();
+    WebAppTests.flushOutput(injector);
+
+    ahsViewInstance.set(YarnWebParams.APP_STATE,
+        YarnApplicationState.FAILED.toString());
+    ahsViewInstance.render();
+    WebAppTests.flushOutput(injector);
+
+    ahsViewInstance.set(YarnWebParams.APP_STATE, StringHelper.cjoin(
+        YarnApplicationState.FAILED.toString(), YarnApplicationState.KILLED));
+    ahsViewInstance.render();
+    WebAppTests.flushOutput(injector);
+  }
+
+  @Test
+  public void testAppPage() throws Exception {
+    Injector injector =
+        WebAppTests.createMockInjector(ApplicationContext.class,
+            mockApplicationHistoryManager(1, 5, 1));
+    AppPage appPageInstance = injector.getInstance(AppPage.class);
+
+    appPageInstance.render();
+    WebAppTests.flushOutput(injector);
+
+    appPageInstance.set(YarnWebParams.APPLICATION_ID,
+        ApplicationId.newInstance(0, 1).toString());
+    appPageInstance.render();
+    WebAppTests.flushOutput(injector);
+  }
+
+  @Test
+  public void testAppAttemptPage() throws Exception {
+    Injector injector =
+        WebAppTests.createMockInjector(ApplicationContext.class,
+            mockApplicationHistoryManager(1, 1, 5));
+    AppAttemptPage appAttemptPageInstance =
+        injector.getInstance(AppAttemptPage.class);
+
+    appAttemptPageInstance.render();
+    WebAppTests.flushOutput(injector);
+
+    appAttemptPageInstance.set(YarnWebParams.APPLICATION_ATTEMPT_ID,
+        ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1)
+            .toString());
+    appAttemptPageInstance.render();
+    WebAppTests.flushOutput(injector);
+  }
+
+  @Test
+  public void testContainerPage() throws Exception {
+    Injector injector =
+        WebAppTests.createMockInjector(ApplicationContext.class,
+            mockApplicationHistoryManager(1, 1, 1));
+    ContainerPage containerPageInstance =
+        injector.getInstance(ContainerPage.class);
+
+    containerPageInstance.render();
+    WebAppTests.flushOutput(injector);
+
+    containerPageInstance.set(
+        YarnWebParams.CONTAINER_ID,
+        ContainerId
+            .newInstance(
+                ApplicationAttemptId.newInstance(
+                    ApplicationId.newInstance(0, 1), 1), 1).toString());
+    containerPageInstance.render();
+    WebAppTests.flushOutput(injector);
+  }
+
+  private ApplicationHistoryManager mockApplicationHistoryManager(
+      int numApps, int numAppAttempts, int numContainers) throws Exception {
+    ApplicationHistoryManager ahManager =
+        new MockApplicationHistoryManagerImpl();
+    for (int i = 1; i <= numApps; ++i) {
+      ApplicationId appId = ApplicationId.newInstance(0, i);
+      writeApplicationStartData(appId);
+      for (int j = 1; j <= numAppAttempts; ++j) {
+        ApplicationAttemptId appAttemptId =
+            ApplicationAttemptId.newInstance(appId, j);
+        writeApplicationAttemptStartData(appAttemptId);
+        for (int k = 1; k <= numContainers; ++k) {
+          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          writeContainerStartData(containerId);
+          writeContainerFinishData(containerId);
+        }
+        writeApplicationAttemptFinishData(appAttemptId);
+      }
+      writeApplicationFinishData(appId);
+    }
+    return ahManager;
+  }
+
+  private class MockApplicationHistoryManagerImpl extends
+      ApplicationHistoryManagerImpl {
+
+    public MockApplicationHistoryManagerImpl() {
+      super();
+      init(new YarnConfiguration());
+      start();
+    }
+
+    @Override
+    protected ApplicationHistoryStore createApplicationHistoryStore(
+        Configuration conf) {
+      return store;
+    }
+  };
+
+}

+ 121 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java

@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.api;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+
+@Public
+@Unstable
+public interface ApplicationContext {
+  /**
+   * This method returns Application {@link ApplicationReport} for the specified
+   * {@link ApplicationId}.
+   * 
+   * @return {@link ApplicationReport} for the ApplicationId.
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  ApplicationReport getApplication(ApplicationId appId) throws IOException;
+
+  /**
+   * This method returns all Application {@link ApplicationReport}s
+   * 
+   * @return map {@link ApplicationId, @link ApplicationReport}s.
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  Map<ApplicationId, ApplicationReport> getAllApplications() throws IOException;
+
+  /**
+   * Application can have multiple application attempts
+   * {@link ApplicationAttemptReport}. This method returns the all
+   * {@link ApplicationAttemptReport}s for the Application.
+   * 
+   * @return all {@link ApplicationAttemptReport}s for the Application.
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  Map<ApplicationAttemptId, ApplicationAttemptReport> getApplicationAttempts(
+      ApplicationId appId) throws IOException;
+
+  /**
+   * This method returns {@link ApplicationAttemptReport} for specified
+   * {@link ApplicationId}.
+   * 
+   * @param {@link ApplicationAttemptId}
+   * @return {@link ApplicationAttemptReport} for ApplicationAttemptId
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  ApplicationAttemptReport getApplicationAttempt(
+      ApplicationAttemptId appAttemptId) throws IOException;
+
+  /**
+   * This method returns {@link ContainerReport} for specified
+   * {@link ContainerId}.
+   * 
+   * @param {@link ContainerId}
+   * @return {@link ContainerReport} for ContainerId
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  ContainerReport getContainer(ContainerId containerId) throws IOException;
+
+  /**
+   * This method returns {@link ContainerReport} for specified
+   * {@link ApplicationAttemptId}.
+   * 
+   * @param {@link ApplicationAttemptId}
+   * @return {@link ContainerReport} for ApplicationAttemptId
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
+      throws IOException;
+
+  /**
+   * This method returns Map{@link ContainerId,@link ContainerReport} for
+   * specified {@link ApplicationAttemptId}.
+   * 
+   * @param {@link ApplicationAttemptId}
+   * @return Map{@link ContainerId, @link ContainerReport} for
+   *         ApplicationAttemptId
+   * @throws {@link IOException}
+   */
+  @Public
+  @Unstable
+  Map<ContainerId, ContainerReport> getContainers(
+      ApplicationAttemptId appAttemptId) throws IOException;
+}

+ 160 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java

@@ -0,0 +1,160 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.server.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ATTEMPT_ID;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+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 org.apache.hadoop.yarn.webapp.view.InfoBlock;
+
+import com.google.inject.Inject;
+
+public class AppAttemptBlock extends HtmlBlock {
+
+  private static final Log LOG = LogFactory.getLog(AppAttemptBlock.class);
+  private final ApplicationContext appContext;
+
+  @Inject
+  public AppAttemptBlock(ApplicationContext appContext) {
+    this.appContext = appContext;
+  }
+
+  @Override
+  protected void render(Block html) {
+    String attemptid = $(APPLICATION_ATTEMPT_ID);
+    if (attemptid.isEmpty()) {
+      puts("Bad request: requires application attempt ID");
+      return;
+    }
+
+    ApplicationAttemptId appAttemptId = null;
+    try {
+      appAttemptId = ConverterUtils.toApplicationAttemptId(attemptid);
+    } catch (IllegalArgumentException e) {
+      puts("Invalid application attempt ID: " + attemptid);
+      return;
+    }
+
+    ApplicationAttemptReport appAttemptReport;
+    try {
+      appAttemptReport = appContext.getApplicationAttempt(appAttemptId);
+    } catch (IOException e) {
+      String message =
+          "Failed to read the application attempt " + appAttemptId + ".";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+    if (appAttemptReport == null) {
+      puts("Application Attempt not found: " + attemptid);
+      return;
+    }
+    AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
+
+    setTitle(join("Application Attempt ", attemptid));
+
+    String node = "N/A";
+    if (appAttempt.getHost() != null && appAttempt.getRpcPort() >= 0
+        && appAttempt.getRpcPort() < 65536) {
+      node = appAttempt.getHost() + ":" + appAttempt.getRpcPort();
+    }
+    info("Application Attempt Overview").
+        _("State", appAttempt.getAppAttemptState()).
+        _("Master Container",
+            appAttempt.getAmContainerId() == null ? "#" : root_url("container",
+                appAttempt.getAmContainerId()),
+            String.valueOf(appAttempt.getAmContainerId())).
+        _("Node:", node).
+        _("Tracking URL:",
+            appAttempt.getTrackingUrl() == null ? "#" : root_url(appAttempt
+                .getTrackingUrl()), "History").
+        _("Diagnostics Info:", appAttempt.getDiagnosticsInfo());
+
+    html._(InfoBlock.class);
+
+    Collection<ContainerReport> containers;
+    try {
+      containers = appContext.getContainers(appAttemptId).values();
+    } catch (IOException e) {
+      html.p()._("Sorry, Failed to get containers for application attempt"
+          + attemptid + ".")._();
+      return;
+    }
+    
+    // Container Table
+    TBODY<TABLE<Hamlet>> tbody = html.
+        table("#containers").
+        thead().
+        tr().
+        th(".id", "Container ID").
+        th(".node", "Node").
+        th(".exitstatus", "Container Exit Status").
+        th(".logs", "Logs")._()._().
+        tbody();
+
+    StringBuilder containersTableData = new StringBuilder("[\n");
+    for (ContainerReport containerReport : containers) {
+      ContainerInfo container = new ContainerInfo(containerReport);
+      // ConatinerID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
+      containersTableData
+          .append("[\"<a href='")
+          .append(url("container", container.getContainerId()))
+          .append("'>")
+          .append(container.getContainerId())
+          .append("</a>\",\"<a href='")
+          .append(container.getAssignedNodeId())
+          .append("'>")
+          .append(StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(container.getAssignedNodeId())))
+          .append("</a>\",\"")
+          .append(container.getContainerExitStatus())
+          .append("\",\"<a href='")
+          .append(container.getLogUrl() == null ? "#" : url(container.getLogUrl()))
+          .append("'>")
+          .append(container.getLogUrl() == null ? "N/A" : "Logs")
+          .append("</a>\"],\n");
+    }
+    if (containersTableData.charAt(containersTableData.length() - 2) == ',') {
+      containersTableData.delete(containersTableData.length() - 2,
+          containersTableData.length() - 1);
+    }
+    containersTableData.append("]");
+    html.script().$type("text/javascript").
+        _("var containersTableData=" + containersTableData)._();
+
+
+    tbody._()._();
+  }
+}

+ 188 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java

@@ -0,0 +1,188 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ID;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.util.Apps;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+
+import com.google.inject.Inject;
+
+public class AppBlock extends HtmlBlock {
+
+  protected ApplicationContext appContext;
+
+  @Inject
+  AppBlock(ApplicationContext appContext, ViewContext ctx) {
+    super(ctx);
+    this.appContext = appContext;
+  }
+
+  @Override
+  protected void render(Block html) {
+    String aid = $(APPLICATION_ID);
+    if (aid.isEmpty()) {
+      puts("Bad request: requires Application ID");
+      return;
+    }
+
+    ApplicationId appID = null;
+    try {
+      appID = Apps.toAppID(aid);
+    } catch (Exception e) {
+      puts("Invalid Application ID: " + aid);
+      return;
+    }
+
+    ApplicationReport appReport;
+    try {
+      appReport = appContext.getApplication(appID);
+    } catch (IOException e) {
+      String message =
+          "Failed to read the application " + appID + ".";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+    if (appReport == null) {
+      puts("Application not found: " + aid);
+      return;
+    }
+    AppInfo app = new AppInfo(appReport);
+
+    setTitle(join("Application ", aid));
+
+    info("Application Overview").
+        _("User:", app.getUser()).
+        _("Name:", app.getName()).
+        _("Application Type:", app.getType()).
+        _("State:", app.getAppState()).
+        _("FinalStatus:", app.getFinalAppStatus()).
+        _("Started:", Times.format(app.getStartedTime())).
+        _("Elapsed:", StringUtils.formatTime(
+            Times.elapsed(app.getStartedTime(), app.getFinishedTime()))).
+        _("Tracking URL:",
+            app.getTrackingUrl() == null ? "#" : root_url(app.getTrackingUrl()),
+            "History").
+        _("Diagnostics:", app.getDiagnosticsInfo());
+
+    html._(InfoBlock.class);
+
+    Collection<ApplicationAttemptReport> attempts;
+    try {
+      attempts = appContext.getApplicationAttempts(appID).values();
+    } catch (IOException e) {
+      String message =
+          "Failed to read the attempts of the application " + appID + ".";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+
+    // Application Attempt Table
+    TBODY<TABLE<Hamlet>> tbody = html.
+        table("#attempts").
+        thead().
+        tr().
+        th(".id", "Attempt ID").
+        th(".started", "Started").
+        th(".node", "Node").
+        th(".logs", "Logs")._()._().
+        tbody();
+
+    StringBuilder attemptsTableData = new StringBuilder("[\n");
+    for (ApplicationAttemptReport appAttemptReport : attempts) {
+      AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
+      ContainerReport containerReport;
+      try {
+        containerReport = appContext.getAMContainer(appAttemptReport
+            .getApplicationAttemptId());
+      } catch (IOException e) {
+        String message =
+            "Failed to read the AM container of the application attempt "
+                + appAttemptReport.getApplicationAttemptId() + ".";
+        LOG.error(message, e);
+        html.p()._(message)._();
+        return;
+      }
+      long startTime = Long.MAX_VALUE;
+      String logsLink = null;
+      if (containerReport != null) {
+        ContainerInfo container = new ContainerInfo(containerReport);
+        startTime = container.getStartedTime();
+        logsLink = container.getLogUrl();
+      }
+      String nodeLink = null;
+      if (appAttempt.getHost() != null && appAttempt.getRpcPort() >= 0
+          && appAttempt.getRpcPort() < 65536) {
+        nodeLink = appAttempt.getHost() + ":" + appAttempt.getRpcPort();
+      }
+      // AppAttemptID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
+      attemptsTableData
+          .append("[\"<a href='")
+          .append(url("appattempt", appAttempt.getAppAttemptId()))
+          .append("'>")
+          .append(appAttempt.getAppAttemptId())
+          .append("</a>\",\"")
+          .append(startTime)
+          .append("\",\"<a href='")
+          .append(nodeLink == null ? "#" : url(HttpConfig.getSchemePrefix(),
+              nodeLink))
+          .append("'>")
+          .append(nodeLink == null ? "N/A" : StringEscapeUtils.escapeJavaScript(
+              StringEscapeUtils.escapeHtml(nodeLink)))
+          .append("</a>\",\"<a href='")
+          .append(logsLink == null ? "#" : url(logsLink))
+          .append("'>")
+          .append(nodeLink == null ? "N/A" : "Logs")
+          .append("</a>\"],\n");
+    }
+    if (attemptsTableData.charAt(attemptsTableData.length() - 2) == ',') {
+      attemptsTableData.delete(attemptsTableData.length() - 2,
+          attemptsTableData.length() - 1);
+    }
+    attemptsTableData.append("]");
+    html.script().$type("text/javascript").
+        _("var attemptsTableData=" + attemptsTableData)._();
+
+
+    tbody._()._();
+  }
+}

+ 150 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java

@@ -0,0 +1,150 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.APP_STATE;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR;
+import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+
+import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
+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 com.google.inject.Inject;
+
+public class AppsBlock extends HtmlBlock {
+
+  protected ApplicationContext appContext;
+
+  @Inject
+  AppsBlock(ApplicationContext appContext, ViewContext ctx) {
+    super(ctx);
+    this.appContext = appContext;
+  }
+
+  @Override
+  public void render(Block html) {
+    setTitle("Applications");
+
+    TBODY<TABLE<Hamlet>> tbody = html.
+        table("#apps").
+        thead().
+        tr().
+        th(".id", "ID").
+        th(".user", "User").
+        th(".name", "Name").
+        th(".type", "Application Type").
+        th(".queue", "Queue").
+        th(".starttime", "StartTime").
+        th(".finishtime", "FinishTime").
+        th(".state", "State").
+        th(".finalstatus", "FinalStatus").
+        th(".progress", "Progress").
+        th(".ui", "Tracking UI")._()._().
+        tbody();
+    Collection<YarnApplicationState> reqAppStates = null;
+    String reqStateString = $(APP_STATE);
+    if (reqStateString != null && !reqStateString.isEmpty()) {
+      String[] appStateStrings = reqStateString.split(",");
+      reqAppStates = new HashSet<YarnApplicationState>(appStateStrings.length);
+      for (String stateString : appStateStrings) {
+        reqAppStates.add(YarnApplicationState.valueOf(stateString));
+      }
+    }
+
+    Collection<ApplicationReport> appReports;
+    try {
+      appReports = appContext.getAllApplications().values();
+    } catch (IOException e) {
+      String message =
+          "Failed to read the applications.";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+    StringBuilder appsTableData = new StringBuilder("[\n");
+    for (ApplicationReport appReport : appReports) {
+      if (reqAppStates != null
+          && !reqAppStates.contains(appReport.getYarnApplicationState())) {
+        continue;
+      }
+      AppInfo app = new AppInfo(appReport);
+      String percent = String.format("%.1f", app.getProgress());
+      // AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
+      appsTableData
+          .append("[\"<a href='")
+          .append(url("app", app.getAppId()))
+          .append("'>")
+          .append(app.getAppId())
+          .append("</a>\",\"")
+          .append(
+              StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
+                  app.getUser())))
+          .append("\",\"")
+          .append(
+              StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
+                  app.getName())))
+          .append("\",\"")
+          .append(
+              StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
+                  app.getType())))
+          .append("\",\"")
+          .append(
+              StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
+                  app.getQueue()))).append("\",\"")
+          .append(app.getStartedTime()).append("\",\"")
+          .append(app.getFinishedTime()).append("\",\"")
+          .append(app.getAppState()).append("\",\"")
+          .append(app.getFinalAppStatus()).append("\",\"")
+          // Progress bar
+          .append("<br title='").append(percent)
+          .append("'> <div class='").append(C_PROGRESSBAR).append("' title='")
+          .append(join(percent, '%')).append("'> ").append("<div class='")
+          .append(C_PROGRESSBAR_VALUE).append("' style='")
+          .append(join("width:", percent, '%')).append("'> </div> </div>")
+          .append("\",\"<a href='");
+
+      String trackingURL = app.getTrackingUrl() == null ? "#" : app.getTrackingUrl();
+
+      appsTableData.append(trackingURL).append("'>")
+          .append("History").append("</a>\"],\n");
+
+    }
+    if (appsTableData.charAt(appsTableData.length() - 2) == ',') {
+      appsTableData.delete(appsTableData.length() - 2,
+          appsTableData.length() - 1);
+    }
+    appsTableData.append("]");
+    html.script().$type("text/javascript").
+        _("var appsTableData=" + appsTableData)._();
+
+    tbody._()._();
+  }
+}

+ 100 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.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.yarn.server.webapp;
+
+import static org.apache.hadoop.yarn.util.StringHelper.join;
+import static org.apache.hadoop.yarn.webapp.YarnWebParams.CONTAINER_ID;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.server.webapp.dao.ContainerInfo;
+import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.hadoop.yarn.util.Times;
+import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
+import org.apache.hadoop.yarn.webapp.view.InfoBlock;
+
+import com.google.inject.Inject;
+
+public class ContainerBlock extends HtmlBlock {
+
+  private static final Log LOG = LogFactory.getLog(ContainerBlock.class);
+  private final ApplicationContext appContext;
+
+  @Inject
+  public ContainerBlock(ApplicationContext appContext, ViewContext ctx) {
+    super(ctx);
+    this.appContext = appContext;
+  }
+
+  @Override
+  protected void render(Block html) {
+    String containerid = $(CONTAINER_ID);
+    if (containerid.isEmpty()) {
+      puts("Bad request: requires container ID");
+      return;
+    }
+
+    ContainerId containerId = null;
+    try {
+      containerId = ConverterUtils.toContainerId(containerid);
+    } catch (IllegalArgumentException e) {
+      puts("Invalid container ID: " + containerid);
+      return;
+    }
+
+    ContainerReport containerReport;
+    try {
+      containerReport = appContext.getContainer(containerId);
+    } catch (IOException e) {
+      String message = "Failed to read the container " + containerid + ".";
+      LOG.error(message, e);
+      html.p()._(message)._();
+      return;
+    }
+    if (containerReport == null) {
+      puts("Container not found: " + containerid);
+      return;
+    }
+    ContainerInfo container = new ContainerInfo(containerReport);
+
+    setTitle(join("Container ", containerid));
+
+    info("Container Overview").
+    _("State:", container.getContainerState()).
+    _("Exit Status:", container.getContainerExitStatus()).
+    _("Node:", container.getAssignedNodeId()).
+    _("Priority:", container.getPriority()).
+    _("Started:", Times.format(container.getStartedTime())).
+    _("Elapsed:", StringUtils.formatTime(
+        Times.elapsed(container.getStartedTime(),
+            container.getFinishedTime()))).
+    _("Resource:", container.getAllocatedMB() + " Memory, " +
+        container.getAllocatedVCores() + " VCores").
+    _("Logs:", container.getLogUrl() == null ?
+        "#" : root_url(container.getLogUrl()), container.getLogUrl()).
+    _("Diagnostics:", container.getDiagnosticsInfo());
+
+    html._(InfoBlock.class);
+  }
+}

+ 84 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppAttemptInfo.java

@@ -0,0 +1,84 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
+
+@XmlRootElement(name = "appattempt")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AppAttemptInfo {
+
+  protected String appAttemptId;
+  protected String host;
+  protected int rpcPort;
+  protected String trackingUrl;
+  protected String diagnosticsInfo;
+  protected YarnApplicationAttemptState appAttemptState;
+  protected String amContainerId;
+
+  public AppAttemptInfo() {
+    // JAXB needs this
+  }
+
+  public AppAttemptInfo(ApplicationAttemptReport appAttempt) {
+    appAttemptId = appAttempt.getApplicationAttemptId().toString();
+    host = appAttempt.getHost();
+    rpcPort = appAttempt.getRpcPort();
+    trackingUrl = appAttempt.getTrackingUrl();
+    diagnosticsInfo = appAttempt.getDiagnostics();
+    appAttemptState = appAttempt.getYarnApplicationAttemptState();
+    if (appAttempt.getAMContainerId() != null) {
+      amContainerId = appAttempt.getAMContainerId().toString();
+    }
+  }
+
+  public String getAppAttemptId() {
+    return appAttemptId;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getRpcPort() {
+    return rpcPort;
+  }
+
+  public String getTrackingUrl() {
+    return trackingUrl;
+  }
+
+  public String getDiagnosticsInfo() {
+    return diagnosticsInfo;
+  }
+
+  public YarnApplicationAttemptState getAppAttemptState() {
+    return appAttemptState;
+  }
+
+  public String getAmContainerId() {
+    return amContainerId;
+  }
+
+}

+ 169 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java

@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "app")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class AppInfo {
+
+  protected String appId;
+  protected String currentAppAttemptId;
+  protected String user;
+  protected String name;
+  protected String queue;
+  protected String type;
+  protected String host;
+  protected int rpcPort;
+  protected YarnApplicationState appState;
+  protected float progress;
+  protected String diagnosticsInfo;
+  protected String originalTrackingUrl;
+  protected String trackingUrl;
+  protected FinalApplicationStatus finalAppStatus;
+  protected long submittedTime;
+  protected long startedTime;
+  protected long finishedTime;
+  protected long elapsedTime;
+  protected int allocatedMB;
+  protected int allocatedVCores;
+
+  public AppInfo() {
+    // JAXB needs this
+  }
+
+  public AppInfo(ApplicationReport app) {
+    appId = app.getApplicationId().toString();
+    if (app.getCurrentApplicationAttemptId() != null) {
+      currentAppAttemptId = app.getCurrentApplicationAttemptId().toString();
+    }
+    user = app.getUser();
+    queue = app.getQueue();
+    name = app.getName();
+    type = app.getApplicationType();
+    host = app.getHost();
+    rpcPort = app.getRpcPort();
+    appState = app.getYarnApplicationState();
+    diagnosticsInfo = app.getDiagnostics();
+    trackingUrl = app.getTrackingUrl();
+    originalTrackingUrl = app.getOriginalTrackingUrl();
+    submittedTime = app.getStartTime();
+    startedTime = app.getStartTime();
+    finishedTime = app.getFinishTime();
+    elapsedTime = Times.elapsed(startedTime, finishedTime);
+    finalAppStatus = app.getFinalApplicationStatus();
+    ApplicationResourceUsageReport usage =
+        app.getApplicationResourceUsageReport();
+    if (usage != null) {
+      allocatedMB = usage.getUsedResources().getMemory();
+      allocatedVCores = usage.getUsedResources().getVirtualCores();
+    }
+    progress = app.getProgress();
+  }
+
+  public String getAppId() {
+    return appId;
+  }
+
+  public String getCurrentAppAttemptId() {
+    return currentAppAttemptId;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getQueue() {
+    return queue;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public int getRpcPort() {
+    return rpcPort;
+  }
+
+  public YarnApplicationState getAppState() {
+    return appState;
+  }
+
+  public float getProgress() {
+    return progress;
+  }
+
+  public String getDiagnosticsInfo() {
+    return diagnosticsInfo;
+  }
+
+  public String getOriginalTrackingUrl() {
+    return originalTrackingUrl;
+  }
+
+  public String getTrackingUrl() {
+    return trackingUrl;
+  }
+
+  public FinalApplicationStatus getFinalAppStatus() {
+    return finalAppStatus;
+  }
+
+  public long getSubmittedTime() {
+    return submittedTime;
+  }
+
+  public long getStartedTime() {
+    return startedTime;
+  }
+
+  public long getFinishedTime() {
+    return finishedTime;
+  }
+
+  public long getElapsedTime() {
+    return elapsedTime;
+  }
+
+  public int getAllocatedMB() {
+    return allocatedMB;
+  }
+
+  public int getAllocatedVCores() {
+    return allocatedVCores;
+  }
+
+}

+ 118 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/ContainerInfo.java

@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.webapp.dao;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.ContainerState;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.util.Times;
+
+@XmlRootElement(name = "container")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class ContainerInfo {
+
+  protected String containerId;
+  protected int allocatedMB;
+  protected int allocatedVCores;
+  protected String assignedNodeId;
+  protected Priority priority;
+  protected long startedTime;
+  protected long finishedTime;
+  protected long elapsedTime;
+  protected String diagnosticsInfo;
+  protected String logUrl;
+  protected int containerExitStatus;
+  protected ContainerState containerState;
+
+  public ContainerInfo() {
+    // JAXB needs this
+  }
+
+  public ContainerInfo(ContainerReport container) {
+    containerId = container.getContainerId().toString();
+    if (container.getAllocatedResource() != null) {
+      allocatedMB = container.getAllocatedResource().getMemory();
+      allocatedVCores = container.getAllocatedResource().getVirtualCores();
+    }
+    if (container.getAssignedNode() != null) {
+      assignedNodeId = container.getAssignedNode().toString();
+    }
+    priority = container.getPriority();
+    startedTime = container.getStartTime();
+    finishedTime = container.getFinishTime();
+    elapsedTime = Times.elapsed(startedTime, finishedTime);
+    diagnosticsInfo = container.getDiagnosticsInfo();
+    logUrl = container.getLogUrl();
+    containerExitStatus = container.getContainerExitStatus();
+    containerState = container.getContainerState();
+  }
+
+  public String getContainerId() {
+    return containerId;
+  }
+
+  public int getAllocatedMB() {
+    return allocatedMB;
+  }
+
+  public int getAllocatedVCores() {
+    return allocatedVCores;
+  }
+
+  public String getAssignedNodeId() {
+    return assignedNodeId;
+  }
+
+  public Priority getPriority() {
+    return priority;
+  }
+
+  public long getStartedTime() {
+    return startedTime;
+  }
+
+  public long getFinishedTime() {
+    return finishedTime;
+  }
+
+  public long getElapsedTime() {
+    return elapsedTime;
+  }
+
+  public String getDiagnosticsInfo() {
+    return diagnosticsInfo;
+  }
+
+  public String getLogUrl() {
+    return logUrl;
+  }
+
+  public int getContainerExitStatus() {
+    return containerExitStatus;
+  }
+
+  public ContainerState getContainerState() {
+    return containerState;
+  }
+
+}