You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/03/13 00:04:21 UTC

[02/49] hadoop git commit: YARN-1809. Synchronize RM and TimeLineServer Web-UIs. Contributed by Zhijie Shen and Xuan Gong

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java
index 7bac6f2..2cd7580 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java
+++ b/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,15 +20,16 @@ package org.apache.hadoop.yarn.server.applicationhistoryservice.webapp;
 
 import static org.apache.hadoop.yarn.webapp.Params.TITLE;
 import static org.mockito.Mockito.mock;
-import org.junit.Assert;
 
+import org.junit.Assert;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 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.ApplicationHistoryClientService;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManager;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerImpl;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore;
@@ -68,8 +69,8 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
   @Test
   public void testView() throws Exception {
     Injector injector =
-        WebAppTests.createMockInjector(ApplicationContext.class,
-          mockApplicationHistoryManager(5, 1, 1));
+        WebAppTests.createMockInjector(ApplicationBaseProtocol.class,
+          mockApplicationHistoryClientService(5, 1, 1));
     AHSView ahsViewInstance = injector.getInstance(AHSView.class);
 
     ahsViewInstance.render();
@@ -89,8 +90,8 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
   @Test
   public void testAppPage() throws Exception {
     Injector injector =
-        WebAppTests.createMockInjector(ApplicationContext.class,
-          mockApplicationHistoryManager(1, 5, 1));
+        WebAppTests.createMockInjector(ApplicationBaseProtocol.class,
+          mockApplicationHistoryClientService(1, 5, 1));
     AppPage appPageInstance = injector.getInstance(AppPage.class);
 
     appPageInstance.render();
@@ -105,8 +106,8 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
   @Test
   public void testAppAttemptPage() throws Exception {
     Injector injector =
-        WebAppTests.createMockInjector(ApplicationContext.class,
-          mockApplicationHistoryManager(1, 1, 5));
+        WebAppTests.createMockInjector(ApplicationBaseProtocol.class,
+          mockApplicationHistoryClientService(1, 1, 5));
     AppAttemptPage appAttemptPageInstance =
         injector.getInstance(AppAttemptPage.class);
 
@@ -123,8 +124,8 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
   @Test
   public void testContainerPage() throws Exception {
     Injector injector =
-        WebAppTests.createMockInjector(ApplicationContext.class,
-          mockApplicationHistoryManager(1, 1, 1));
+        WebAppTests.createMockInjector(ApplicationBaseProtocol.class,
+          mockApplicationHistoryClientService(1, 1, 1));
     ContainerPage containerPageInstance =
         injector.getInstance(ContainerPage.class);
 
@@ -141,10 +142,12 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
     WebAppTests.flushOutput(injector);
   }
 
-  ApplicationHistoryManager mockApplicationHistoryManager(int numApps,
+  ApplicationHistoryClientService mockApplicationHistoryClientService(int numApps,
       int numAppAttempts, int numContainers) throws Exception {
     ApplicationHistoryManager ahManager =
         new MockApplicationHistoryManagerImpl(store);
+    ApplicationHistoryClientService historyClientService =
+        new ApplicationHistoryClientService(ahManager);
     for (int i = 1; i <= numApps; ++i) {
       ApplicationId appId = ApplicationId.newInstance(0, i);
       writeApplicationStartData(appId);
@@ -161,7 +164,7 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
       }
       writeApplicationFinishData(appId);
     }
-    return ahManager;
+    return historyClientService;
   }
 
   class MockApplicationHistoryManagerImpl extends ApplicationHistoryManagerImpl {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
index 41dda91..913b80d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
@@ -28,13 +28,11 @@ import java.util.Properties;
 import javax.servlet.FilterConfig;
 import javax.servlet.ServletException;
 import javax.ws.rs.core.MediaType;
-import javax.ws.rs.core.Response;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.http.lib.StaticUserWebFilter.StaticUserFilter;
 import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
-import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
 import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -44,7 +42,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
 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.ApplicationHistoryClientService;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryManagerOnTimelineStore;
 import org.apache.hadoop.yarn.server.applicationhistoryservice.TestApplicationHistoryManagerOnTimelineStore;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
@@ -79,7 +77,7 @@ import com.sun.jersey.test.framework.WebAppDescriptor;
 @RunWith(Parameterized.class)
 public class TestAHSWebServices extends JerseyTestBase {
 
-  private static ApplicationHistoryManagerOnTimelineStore historyManager;
+  private static ApplicationHistoryClientService historyClientService;
   private static final String[] USERS = new String[] { "foo" , "bar" };
 
   @BeforeClass
@@ -93,16 +91,23 @@ public class TestAHSWebServices extends JerseyTestBase {
     conf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
     conf.set(YarnConfiguration.YARN_ADMIN_ACL, "foo");
     ApplicationACLsManager appAclsManager = new ApplicationACLsManager(conf);
-    historyManager =
+    ApplicationHistoryManagerOnTimelineStore historyManager =
         new ApplicationHistoryManagerOnTimelineStore(dataManager, appAclsManager);
     historyManager.init(conf);
-    historyManager.start();
+    historyClientService = new ApplicationHistoryClientService(historyManager) {
+      @Override
+      protected void serviceStart() throws Exception {
+        // Do Nothing
+      }
+    };
+    historyClientService.init(conf);
+    historyClientService.start();
   }
 
   @AfterClass
   public static void tearDownClass() throws Exception {
-    if (historyManager != null) {
-      historyManager.stop();
+    if (historyClientService != null) {
+      historyClientService.stop();
     }
   }
 
@@ -118,7 +123,7 @@ public class TestAHSWebServices extends JerseyTestBase {
       bind(JAXBContextResolver.class);
       bind(AHSWebServices.class);
       bind(GenericExceptionHandler.class);
-      bind(ApplicationContext.class).toInstance(historyManager);
+      bind(ApplicationBaseProtocol.class).toInstance(historyClientService);
       serve("/*").with(GuiceContainer.class);
       filter("/*").through(TestSimpleAuthFilter.class);
     }
@@ -372,5 +377,4 @@ public class TestAHSWebServices extends JerseyTestBase {
     assertEquals(ContainerState.COMPLETE.toString(),
       container.getString("containerState"));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java
deleted file mode 100644
index 0e2ffdf..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java
+++ /dev/null
@@ -1,122 +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.api;
-
-import java.io.IOException;
-import java.util.Map;
-
-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.exceptions.YarnException;
-
-public interface ApplicationContext {
-  /**
-   * This method returns Application {@link ApplicationReport} for the specified
-   * {@link ApplicationId}.
-   * 
-   * @param appId
-   * 
-   * @return {@link ApplicationReport} for the ApplicationId.
-   * @throws YarnException
-   * @throws IOException
-   */
-  ApplicationReport getApplication(ApplicationId appId)
-      throws YarnException, IOException;
-
-  /**
-   * This method returns all Application {@link ApplicationReport}s
-   * 
-   * @return map of {@link ApplicationId} to {@link ApplicationReport}s.
-   * @throws YarnException
-   * @throws IOException
-   */
-  Map<ApplicationId, ApplicationReport> getAllApplications()
-      throws YarnException, IOException;
-
-  /**
-   * Application can have multiple application attempts
-   * {@link ApplicationAttemptReport}. This method returns the all
-   * {@link ApplicationAttemptReport}s for the Application.
-   * 
-   * @param appId
-   * 
-   * @return all {@link ApplicationAttemptReport}s for the Application.
-   * @throws YarnException
-   * @throws IOException
-   */
-  Map<ApplicationAttemptId, ApplicationAttemptReport> getApplicationAttempts(
-      ApplicationId appId) throws YarnException, IOException;
-
-  /**
-   * This method returns {@link ApplicationAttemptReport} for specified
-   * {@link ApplicationId}.
-   * 
-   * @param appAttemptId
-   *          {@link ApplicationAttemptId}
-   * @return {@link ApplicationAttemptReport} for ApplicationAttemptId
-   * @throws YarnException
-   * @throws IOException
-   */
-  ApplicationAttemptReport getApplicationAttempt(
-      ApplicationAttemptId appAttemptId) throws YarnException, IOException;
-
-  /**
-   * This method returns {@link ContainerReport} for specified
-   * {@link ContainerId}.
-   * 
-   * @param containerId
-   *          {@link ContainerId}
-   * @return {@link ContainerReport} for ContainerId
-   * @throws YarnException
-   * @throws IOException
-   */
-  ContainerReport getContainer(ContainerId containerId)
-      throws YarnException, IOException;
-
-  /**
-   * This method returns {@link ContainerReport} for specified
-   * {@link ApplicationAttemptId}.
-   * 
-   * @param appAttemptId
-   *          {@link ApplicationAttemptId}
-   * @return {@link ContainerReport} for ApplicationAttemptId
-   * @throws YarnException
-   * @throws IOException
-   */
-  ContainerReport getAMContainer(ApplicationAttemptId appAttemptId)
-      throws YarnException, IOException;
-
-  /**
-   * This method returns Map of {@link ContainerId} to {@link ContainerReport}
-   * for specified {@link ApplicationAttemptId}.
-   * 
-   * @param appAttemptId
-   *          {@link ApplicationAttemptId}
-   * @return Map of {@link ContainerId} to {@link ContainerReport} for
-   *         ApplicationAttemptId
-   * @throws YarnException
-   * @throws IOException
-   */
-  Map<ContainerId, ContainerReport> getContainers(
-      ApplicationAttemptId appAttemptId) throws YarnException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
index 4a02892..ea33f4f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppAttemptBlock.java
@@ -27,10 +27,14 @@ import org.apache.commons.lang.StringEscapeUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+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.api.records.YarnApplicationAttemptState;
 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;
@@ -45,11 +49,12 @@ import com.google.inject.Inject;
 public class AppAttemptBlock extends HtmlBlock {
 
   private static final Log LOG = LogFactory.getLog(AppAttemptBlock.class);
-  private final ApplicationContext appContext;
+  protected ApplicationBaseProtocol appBaseProt;
 
   @Inject
-  public AppAttemptBlock(ApplicationContext appContext) {
-    this.appContext = appContext;
+  public AppAttemptBlock(ApplicationBaseProtocol appBaseProt, ViewContext ctx) {
+    super(ctx);
+    this.appBaseProt = appBaseProt;
   }
 
   @Override
@@ -68,18 +73,22 @@ public class AppAttemptBlock extends HtmlBlock {
       return;
     }
 
-    final ApplicationAttemptId appAttemptIdFinal = appAttemptId;
     UserGroupInformation callerUGI = getCallerUGI();
-    ApplicationAttemptReport appAttemptReport;
+    ApplicationAttemptReport appAttemptReport = null;
     try {
+      final GetApplicationAttemptReportRequest request =
+          GetApplicationAttemptReportRequest.newInstance(appAttemptId);
       if (callerUGI == null) {
-        appAttemptReport = appContext.getApplicationAttempt(appAttemptId);
+        appAttemptReport =
+            appBaseProt.getApplicationAttemptReport(request)
+              .getApplicationAttemptReport();
       } else {
         appAttemptReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationAttemptReport> () {
           @Override
           public ApplicationAttemptReport run() throws Exception {
-            return appContext.getApplicationAttempt(appAttemptIdFinal);
+            return appBaseProt.getApplicationAttemptReport(request)
+                .getApplicationAttemptReport();
           }
         });
       }
@@ -90,10 +99,35 @@ public class AppAttemptBlock extends HtmlBlock {
       html.p()._(message)._();
       return;
     }
+
     if (appAttemptReport == null) {
       puts("Application Attempt not found: " + attemptid);
       return;
     }
+
+    boolean exceptionWhenGetContainerReports = false;
+    Collection<ContainerReport> containers = null;
+    try {
+      final GetContainersRequest request =
+          GetContainersRequest.newInstance(appAttemptId);
+      if (callerUGI == null) {
+        containers = appBaseProt.getContainers(request).getContainerList();
+      } else {
+        containers = callerUGI.doAs(
+            new PrivilegedExceptionAction<Collection<ContainerReport>> () {
+          @Override
+          public Collection<ContainerReport> run() throws Exception {
+            return  appBaseProt.getContainers(request).getContainerList();
+          }
+        });
+      }
+    } catch (RuntimeException e) {
+      // have this block to suppress the findbugs warning
+      exceptionWhenGetContainerReports = true;
+    } catch (Exception e) {
+      exceptionWhenGetContainerReports = true;
+    }
+
     AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
 
     setTitle(join("Application Attempt ", attemptid));
@@ -104,43 +138,35 @@ public class AppAttemptBlock extends HtmlBlock {
       node = appAttempt.getHost() + ":" + appAttempt.getRpcPort();
     }
     info("Application Attempt Overview")
-      ._("State", appAttempt.getAppAttemptState())
       ._(
-        "Master Container",
-        appAttempt.getAmContainerId() == null ? "#" : root_url("container",
-          appAttempt.getAmContainerId()),
+        "Application Attempt State:",
+        appAttempt.getAppAttemptState() == null ? UNAVAILABLE : appAttempt
+          .getAppAttemptState())
+      ._(
+        "AM Container:",
+        appAttempt.getAmContainerId() == null || containers == null
+            || !hasAMContainer(appAttemptReport.getAMContainerId(), containers)
+            ? 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());
+        appAttempt.getTrackingUrl() == null
+            || appAttempt.getTrackingUrl() == UNAVAILABLE ? null
+            : root_url(appAttempt.getTrackingUrl()),
+        appAttempt.getTrackingUrl() == null
+            || appAttempt.getTrackingUrl() == UNAVAILABLE
+            ? "Unassigned"
+            : appAttempt.getAppAttemptState() == YarnApplicationAttemptState.FINISHED
+                || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.FAILED
+                || appAttempt.getAppAttemptState() == YarnApplicationAttemptState.KILLED
+                ? "History" : "ApplicationMaster")
+      ._("Diagnostics Info:", appAttempt.getDiagnosticsInfo() == null ?
+          "" : appAttempt.getDiagnosticsInfo());
 
     html._(InfoBlock.class);
 
-    Collection<ContainerReport> containers;
-    try {
-      if (callerUGI == null) {
-        containers = appContext.getContainers(appAttemptId).values();
-      } else {
-        containers = callerUGI.doAs(
-            new PrivilegedExceptionAction<Collection<ContainerReport>> () {
-          @Override
-          public Collection<ContainerReport> run() throws Exception {
-            return  appContext.getContainers(appAttemptIdFinal).values();
-          }
-        });
-      }
-    } catch (RuntimeException e) {
-      // have this block to suppress the findbugs warning
-      html
-      .p()
-      ._(
-        "Sorry, Failed to get containers for application attempt" + attemptid
-            + ".")._();
-      return;
-    } catch (Exception e) {
+    if (exceptionWhenGetContainerReports) {
       html
         .p()
         ._(
@@ -166,11 +192,12 @@ public class AppAttemptBlock extends HtmlBlock {
         .append("'>")
         .append(container.getContainerId())
         .append("</a>\",\"<a href='")
-        .append(container.getAssignedNodeId())
+        .append("#") // TODO: replace with node http address (YARN-1884)
         .append("'>")
-        .append(
-          StringEscapeUtils.escapeJavaScript(StringEscapeUtils
-            .escapeHtml(container.getAssignedNodeId()))).append("</a>\",\"")
+        .append(container.getAssignedNodeId() == null ? "N/A" :
+            StringEscapeUtils.escapeJavaScript(StringEscapeUtils
+                .escapeHtml(container.getAssignedNodeId())))
+        .append("</a>\",\"")
         .append(container.getContainerExitStatus()).append("\",\"<a href='")
         .append(container.getLogUrl() == null ?
             "#" : container.getLogUrl()).append("'>")
@@ -187,4 +214,14 @@ public class AppAttemptBlock extends HtmlBlock {
 
     tbody._()._();
   }
+
+  private boolean hasAMContainer(ContainerId containerId,
+      Collection<ContainerReport> containers) {
+    for (ContainerReport container : containers) {
+      if (containerId.equals(container.getContainerId())) {
+        return true;
+      }
+    }
+    return false;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
index 8fa4086..2db88ae 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppBlock.java
@@ -20,24 +20,47 @@ 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 static org.apache.hadoop.yarn.webapp.YarnWebParams.WEB_UI_TYPE;
+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._TH;
 
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
+import java.util.List;
 
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 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.ApplicationResourceUsageReport;
+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.api.records.FinalApplicationStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 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.util.resource.Resources;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
+import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
 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;
@@ -47,16 +70,20 @@ import com.google.inject.Inject;
 
 public class AppBlock extends HtmlBlock {
 
-  protected ApplicationContext appContext;
+  private static final Log LOG = LogFactory.getLog(AppBlock.class);
+  protected ApplicationBaseProtocol appBaseProt;
+  protected Configuration conf;
 
   @Inject
-  AppBlock(ApplicationContext appContext, ViewContext ctx) {
+  AppBlock(ApplicationBaseProtocol appBaseProt, ViewContext ctx, Configuration conf) {
     super(ctx);
-    this.appContext = appContext;
+    this.appBaseProt = appBaseProt;
+    this.conf = conf;
   }
 
   @Override
   protected void render(Block html) {
+    String webUiType = $(WEB_UI_TYPE);
     String aid = $(APPLICATION_ID);
     if (aid.isEmpty()) {
       puts("Bad request: requires Application ID");
@@ -71,18 +98,21 @@ public class AppBlock extends HtmlBlock {
       return;
     }
 
-    final ApplicationId appIDFinal = appID;
     UserGroupInformation callerUGI = getCallerUGI();
-    ApplicationReport appReport;
+    ApplicationReport appReport = null;
     try {
+      final GetApplicationReportRequest request =
+          GetApplicationReportRequest.newInstance(appID);
       if (callerUGI == null) {
-        appReport = appContext.getApplication(appID);
+        appReport =
+            appBaseProt.getApplicationReport(request).getApplicationReport();
       } else {
         appReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationReport> () {
           @Override
           public ApplicationReport run() throws Exception {
-            return appContext.getApplication(appIDFinal);
+            return appBaseProt.getApplicationReport(request)
+                .getApplicationReport();
           }
         });
       }
@@ -92,41 +122,90 @@ public class AppBlock extends HtmlBlock {
       html.p()._(message)._();
       return;
     }
+
     if (appReport == null) {
       puts("Application not found: " + aid);
       return;
     }
+
     AppInfo app = new AppInfo(appReport);
 
     setTitle(join("Application ", aid));
 
+    if (webUiType != null
+        && webUiType.equals(YarnWebParams.RM_WEB_UI)
+        && conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
+          YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED)) {
+      // Application Kill
+      html.div()
+        .button()
+          .$onclick("confirmAction()").b("Kill Application")._()
+          ._();
+
+      StringBuilder script = new StringBuilder();
+      script.append("function confirmAction() {")
+          .append(" b = confirm(\"Are you sure?\");")
+          .append(" if (b == true) {")
+          .append(" $.ajax({")
+          .append(" type: 'PUT',")
+          .append(" url: '/ws/v1/cluster/apps/").append(aid).append("/state',")
+          .append(" contentType: 'application/json',")
+          .append(" data: '{\"state\":\"KILLED\"}',")
+          .append(" dataType: 'json'")
+          .append(" }).done(function(data){")
+          .append(" setTimeout(function(){")
+          .append(" location.href = '/cluster/app/").append(aid).append("';")
+          .append(" }, 1000);")
+          .append(" }).fail(function(data){")
+          .append(" console.log(data);")
+          .append(" });")
+          .append(" }")
+          .append("}");
+
+      html.script().$type("text/javascript")._(script.toString())._();
+    }
+
     info("Application Overview")
       ._("User:", app.getUser())
       ._("Name:", app.getName())
       ._("Application Type:", app.getType())
-      ._("State:", app.getAppState())
-      ._("FinalStatus:", app.getFinalAppStatus())
+      ._("Application Tags:",
+        app.getApplicationTags() == null ? "" : app.getApplicationTags())
+      ._("YarnApplicationState:",
+        app.getAppState() == null ? UNAVAILABLE : clarifyAppState(app
+          .getAppState()))
+      ._("FinalStatus Reported by AM:",
+        clairfyAppFinalStatus(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);
+        app.getTrackingUrl() == null || app.getTrackingUrl() == UNAVAILABLE
+            ? null : root_url(app.getTrackingUrl()),
+        app.getTrackingUrl() == null || app.getTrackingUrl() == UNAVAILABLE
+            ? "Unassigned" : app.getAppState() == YarnApplicationState.FINISHED
+                || app.getAppState() == YarnApplicationState.FAILED
+                || app.getAppState() == YarnApplicationState.KILLED ? "History"
+                : "ApplicationMaster")
+      ._("Diagnostics:",
+          app.getDiagnosticsInfo() == null ? "" : app.getDiagnosticsInfo());
 
     Collection<ApplicationAttemptReport> attempts;
     try {
+      final GetApplicationAttemptsRequest request =
+          GetApplicationAttemptsRequest.newInstance(appID);
       if (callerUGI == null) {
-        attempts = appContext.getApplicationAttempts(appID).values();
+        attempts = appBaseProt.getApplicationAttempts(request)
+            .getApplicationAttemptList();
       } else {
         attempts = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationAttemptReport>> () {
           @Override
           public Collection<ApplicationAttemptReport> run() throws Exception {
-            return appContext.getApplicationAttempts(appIDFinal).values();
+            return appBaseProt.getApplicationAttempts(request)
+                .getApplicationAttemptList();
           }
         });
       }
@@ -138,6 +217,34 @@ public class AppBlock extends HtmlBlock {
       return;
     }
 
+    //TODO:YARN-3284
+    //The preemption metrics will be exposed from ApplicationReport
+    // and ApplicationAttemptReport
+    ApplicationResourceUsageReport usageReport =
+        appReport.getApplicationResourceUsageReport();
+    DIV<Hamlet> pdiv = html.
+        _(InfoBlock.class).
+        div(_INFO_WRAP);
+    info("Application Overview").clear();
+    info("Application Metrics")
+        ._("Total Resource Preempted:",
+          Resources.none()) // TODO: YARN-3284
+        ._("Total Number of Non-AM Containers Preempted:",
+          String.valueOf(0)) // TODO: YARN-3284
+        ._("Total Number of AM Containers Preempted:",
+          String.valueOf(0)) // TODO: YARN-3284
+        ._("Resource Preempted from Current Attempt:",
+          Resources.none()) // TODO: YARN-3284
+        ._("Number of Non-AM Containers Preempted from Current Attempt:",
+          0) // TODO: YARN-3284
+        ._("Aggregate Resource Allocation:",
+          String.format("%d MB-seconds, %d vcore-seconds", usageReport == null
+            ? 0 : usageReport.getMemorySeconds(), usageReport == null ? 0
+            : usageReport.getVcoreSeconds()));
+    pdiv._();
+
+    html._(InfoBlock.class);
+
     // Application Attempt Table
     TBODY<TABLE<Hamlet>> tbody =
         html.table("#attempts").thead().tr().th(".id", "Attempt ID")
@@ -147,18 +254,28 @@ public class AppBlock extends HtmlBlock {
     StringBuilder attemptsTableData = new StringBuilder("[\n");
     for (final ApplicationAttemptReport appAttemptReport : attempts) {
       AppAttemptInfo appAttempt = new AppAttemptInfo(appAttemptReport);
-      ContainerReport containerReport;
+      ContainerReport containerReport = null;
       try {
+        // AM container is always the first container of the attempt
+        final GetContainerReportRequest request =
+            GetContainerReportRequest.newInstance(ContainerId.newContainerId(
+              appAttemptReport.getApplicationAttemptId(), 1));
         if (callerUGI == null) {
-          containerReport = appContext.getAMContainer(appAttemptReport
-              .getApplicationAttemptId());
+          containerReport =
+              appBaseProt.getContainerReport(request).getContainerReport();
         } else {
           containerReport = callerUGI.doAs(
               new PrivilegedExceptionAction<ContainerReport> () {
             @Override
             public ContainerReport run() throws Exception {
-              return appContext.getAMContainer(appAttemptReport
-                  .getApplicationAttemptId());
+              ContainerReport report = null;
+              try {
+                report = appBaseProt.getContainerReport(request)
+                    .getContainerReport();
+              } catch (ContainerNotFoundException ex) {
+                LOG.warn(ex.getMessage());
+              }
+              return report;
             }
           });
         }
@@ -170,7 +287,7 @@ public class AppBlock extends HtmlBlock {
         html.p()._(message)._();
         return;
       }
-      long startTime = Long.MAX_VALUE;
+      long startTime = 0L;
       String logsLink = null;
       if (containerReport != null) {
         ContainerInfo container = new ContainerInfo(containerReport);
@@ -192,14 +309,12 @@ public class AppBlock extends HtmlBlock {
         .append("</a>\",\"")
         .append(startTime)
         .append("\",\"<a href='")
-        .append(
-          nodeLink == null ? "#" : url("//", nodeLink))
+        .append("#") // TODO: replace with node http address (YARN-1884)
         .append("'>")
-        .append(
-          nodeLink == null ? "N/A" : StringEscapeUtils
+        .append(nodeLink == null ? "N/A" : StringEscapeUtils
             .escapeJavaScript(StringEscapeUtils.escapeHtml(nodeLink)))
-        .append("</a>\",\"<a href='")
-        .append(logsLink == null ? "#" : logsLink).append("'>")
+        .append("</a>\",\"<a ")
+        .append(logsLink == null ? "#" : "href='" + logsLink).append("'>")
         .append(logsLink == null ? "N/A" : "Logs").append("</a>\"],\n");
     }
     if (attemptsTableData.charAt(attemptsTableData.length() - 2) == ',') {
@@ -211,5 +326,108 @@ public class AppBlock extends HtmlBlock {
       ._("var attemptsTableData=" + attemptsTableData)._();
 
     tbody._()._();
+
+    createContainerLocalityTable(html); //TODO:YARN-3284
+    createResourceRequestsTable(html, null); //TODO:YARN-3284
+  }
+
+  //TODO: YARN-3284
+  //The containerLocality metrics will be exposed from AttemptReport
+  private void createContainerLocalityTable(Block html) {
+    int totalAllocatedContainers = 0; //TODO: YARN-3284
+    int[][] localityStatistics = new int[0][0];//TODO:YARN-3284
+    DIV<Hamlet> div = html.div(_INFO_WRAP);
+    TABLE<DIV<Hamlet>> table =
+        div.h3(
+          "Total Allocated Containers: "
+              + totalAllocatedContainers).h3("Each table cell"
+            + " represents the number of NodeLocal/RackLocal/OffSwitch containers"
+            + " satisfied by NodeLocal/RackLocal/OffSwitch resource requests.").table(
+          "#containerLocality");
+    table.
+      tr().
+        th(_TH, "").
+        th(_TH, "Node Local Request").
+        th(_TH, "Rack Local Request").
+        th(_TH, "Off Switch Request").
+      _();
+
+    String[] containersType =
+        { "Num Node Local Containers (satisfied by)", "Num Rack Local Containers (satisfied by)",
+            "Num Off Switch Containers (satisfied by)" };
+    boolean odd = false;
+    for (int i = 0; i < localityStatistics.length; i++) {
+      table.tr((odd = !odd) ? _ODD : _EVEN).td(containersType[i])
+        .td(String.valueOf(localityStatistics[i][0]))
+        .td(i == 0 ? "" : String.valueOf(localityStatistics[i][1]))
+        .td(i <= 1 ? "" : String.valueOf(localityStatistics[i][2]))._();
+    }
+    table._();
+    div._();
+  }
+
+  //TODO:YARN-3284
+  //The resource requests metrics will be exposed from attemptReport
+  private void createResourceRequestsTable(Block html, List<ResourceRequest> resouceRequests) {
+    TBODY<TABLE<Hamlet>> tbody =
+        html.table("#ResourceRequests").thead().tr()
+          .th(".priority", "Priority")
+          .th(".resourceName", "ResourceName")
+          .th(".totalResource", "Capability")
+          .th(".numContainers", "NumContainers")
+          .th(".relaxLocality", "RelaxLocality")
+          .th(".nodeLabelExpression", "NodeLabelExpression")._()._().tbody();
+
+    Resource totalResource = Resource.newInstance(0, 0);
+    if (resouceRequests != null) {
+      for (ResourceRequest request : resouceRequests) {
+        if (request.getNumContainers() == 0) {
+          continue;
+        }
+
+        tbody.tr()
+          .td(String.valueOf(request.getPriority()))
+          .td(request.getResourceName())
+          .td(String.valueOf(request.getCapability()))
+          .td(String.valueOf(request.getNumContainers()))
+          .td(String.valueOf(request.getRelaxLocality()))
+          .td(request.getNodeLabelExpression() == null ? "N/A" : request
+              .getNodeLabelExpression())._();
+        if (request.getResourceName().equals(ResourceRequest.ANY)) {
+          Resources.addTo(totalResource,
+            Resources.multiply(request.getCapability(),
+              request.getNumContainers()));
+        }
+      }
+    }
+    html.div().$class("totalResourceRequests")
+      .h3("Total Outstanding Resource Requests: " + totalResource)._();
+    tbody._()._();
+  }
+
+  private String clarifyAppState(YarnApplicationState state) {
+    String ret = state.toString();
+    switch (state) {
+    case NEW:
+      return ret + ": waiting for application to be initialized";
+    case NEW_SAVING:
+      return ret + ": waiting for application to be persisted in state-store.";
+    case SUBMITTED:
+      return ret + ": waiting for application to be accepted by scheduler.";
+    case ACCEPTED:
+      return ret + ": waiting for AM container to be allocated, launched and"
+          + " register with RM.";
+    case RUNNING:
+      return ret + ": AM has registered with RM and started running.";
+    default:
+      return ret;
+    }
+  }
+
+  private String clairfyAppFinalStatus(FinalApplicationStatus status) {
+    if (status == FinalApplicationStatus.UNDEFINED) {
+      return "Application has not completed yet.";
+    }
+    return status.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
index f341cf6..161486d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/AppsBlock.java
@@ -25,13 +25,16 @@ import static org.apache.hadoop.yarn.webapp.view.JQueryUI.C_PROGRESSBAR_VALUE;
 
 import java.security.PrivilegedExceptionAction;
 import java.util.Collection;
-import java.util.HashSet;
+import java.util.EnumSet;
 
 import org.apache.commons.lang.StringEscapeUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
 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;
@@ -42,12 +45,13 @@ import com.google.inject.Inject;
 
 public class AppsBlock extends HtmlBlock {
 
-  protected ApplicationContext appContext;
+  private static final Log LOG = LogFactory.getLog(AppsBlock.class);
+  protected ApplicationBaseProtocol appBaseProt;
 
   @Inject
-  AppsBlock(ApplicationContext appContext, ViewContext ctx) {
+  AppsBlock(ApplicationBaseProtocol appBaseProt, ViewContext ctx) {
     super(ctx);
-    this.appContext = appContext;
+    this.appBaseProt = appBaseProt;
   }
 
   @Override
@@ -61,27 +65,29 @@ public class AppsBlock extends HtmlBlock {
           .th(".finishtime", "FinishTime").th(".state", "State")
           .th(".finalstatus", "FinalStatus").th(".progress", "Progress")
           .th(".ui", "Tracking UI")._()._().tbody();
-    Collection<YarnApplicationState> reqAppStates = null;
+    EnumSet<YarnApplicationState> reqAppStates =
+        EnumSet.noneOf(YarnApplicationState.class);
     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));
+        reqAppStates.add(YarnApplicationState.valueOf(stateString.trim()));
       }
     }
 
     UserGroupInformation callerUGI = getCallerUGI();
-    Collection<ApplicationReport> appReports;
+    Collection<ApplicationReport> appReports = null;
     try {
+      final GetApplicationsRequest request =
+          GetApplicationsRequest.newInstance(reqAppStates);
       if (callerUGI == null) {
-        appReports = appContext.getAllApplications().values();
+        appReports = appBaseProt.getApplications(request).getApplicationList();
       } else {
         appReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
           @Override
           public Collection<ApplicationReport> run() throws Exception {
-            return appContext.getAllApplications().values();
+            return appBaseProt.getApplications(request).getApplicationList();
           }
         });
       }
@@ -93,12 +99,15 @@ public class AppsBlock extends HtmlBlock {
     }
     StringBuilder appsTableData = new StringBuilder("[\n");
     for (ApplicationReport appReport : appReports) {
-      if (reqAppStates != null
+      // TODO: remove the following condition. It is still here because
+      // the history side implementation of ApplicationBaseProtocol
+      // hasn't filtering capability (YARN-1819).
+      if (!reqAppStates.isEmpty()
           && !reqAppStates.contains(appReport.getYarnApplicationState())) {
         continue;
       }
       AppInfo app = new AppInfo(appReport);
-      String percent = String.format("%.1f", app.getProgress() * 100.0F);
+      String percent = String.format("%.1f", app.getProgress());
       // AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
       appsTableData
         .append("[\"<a href='")
@@ -123,7 +132,7 @@ public class AppsBlock extends HtmlBlock {
             .getQueue()))).append("\",\"").append(app.getStartedTime())
         .append("\",\"").append(app.getFinishedTime())
         .append("\",\"")
-        .append(app.getAppState())
+        .append(app.getAppState() == null ? UNAVAILABLE : app.getAppState())
         .append("\",\"")
         .append(app.getFinalAppStatus())
         .append("\",\"")
@@ -132,13 +141,21 @@ public class AppsBlock extends HtmlBlock {
         .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='");
+        .append("'> </div> </div>").append("\",\"<a ");
 
       String trackingURL =
-          app.getTrackingUrl() == null ? "#" : app.getTrackingUrl();
+          app.getTrackingUrl() == null || app.getTrackingUrl() == UNAVAILABLE
+              ? null : app.getTrackingUrl();
 
-      appsTableData.append(trackingURL).append("'>").append("History")
-        .append("</a>\"],\n");
+      String trackingUI =
+          app.getTrackingUrl() == null || app.getTrackingUrl() == UNAVAILABLE
+              ? "Unassigned"
+              : app.getAppState() == YarnApplicationState.FINISHED
+                  || app.getAppState() == YarnApplicationState.FAILED
+                  || app.getAppState() == YarnApplicationState.KILLED
+                  ? "History" : "ApplicationMaster";
+      appsTableData.append(trackingURL == null ? "#" : "href='" + trackingURL)
+        .append("'>").append(trackingUI).append("</a>\"],\n");
 
     }
     if (appsTableData.charAt(appsTableData.length() - 2) == ',') {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
index 2bb48a8..ed50c7a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/ContainerBlock.java
@@ -26,9 +26,10 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
 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;
@@ -40,12 +41,12 @@ import com.google.inject.Inject;
 public class ContainerBlock extends HtmlBlock {
 
   private static final Log LOG = LogFactory.getLog(ContainerBlock.class);
-  private final ApplicationContext appContext;
+  protected ApplicationBaseProtocol appBaseProt;
 
   @Inject
-  public ContainerBlock(ApplicationContext appContext, ViewContext ctx) {
+  public ContainerBlock(ApplicationBaseProtocol appBaseProt, ViewContext ctx) {
     super(ctx);
-    this.appContext = appContext;
+    this.appBaseProt = appBaseProt;
   }
 
   @Override
@@ -64,18 +65,21 @@ public class ContainerBlock extends HtmlBlock {
       return;
     }
 
-    final ContainerId containerIdFinal = containerId;
     UserGroupInformation callerUGI = getCallerUGI();
-    ContainerReport containerReport;
+    ContainerReport containerReport = null;
     try {
+      final GetContainerReportRequest request =
+          GetContainerReportRequest.newInstance(containerId);
       if (callerUGI == null) {
-        containerReport = appContext.getContainer(containerId);
+        containerReport = appBaseProt.getContainerReport(request)
+            .getContainerReport();
       } else {
         containerReport = callerUGI.doAs(
             new PrivilegedExceptionAction<ContainerReport> () {
           @Override
           public ContainerReport run() throws Exception {
-            return appContext.getContainer(containerIdFinal);
+            return appBaseProt.getContainerReport(request)
+                .getContainerReport();
           }
         });
       }
@@ -85,6 +89,7 @@ public class ContainerBlock extends HtmlBlock {
       html.p()._(message)._();
       return;
     }
+
     if (containerReport == null) {
       puts("Container not found: " + containerid);
       return;
@@ -94,7 +99,10 @@ public class ContainerBlock extends HtmlBlock {
     setTitle(join("Container ", containerid));
 
     info("Container Overview")
-      ._("State:", container.getContainerState())
+      ._(
+        "Container State:",
+        container.getContainerState() == null ? UNAVAILABLE : container
+          .getContainerState())
       ._("Exit Status:", container.getContainerExitStatus())
       ._("Node:", container.getAssignedNodeId())
       ._("Priority:", container.getPriority())
@@ -109,7 +117,8 @@ public class ContainerBlock extends HtmlBlock {
             + container.getAllocatedVCores() + " VCores")
       ._("Logs:", container.getLogUrl() == null ? "#" : container.getLogUrl(),
           container.getLogUrl() == null ? "N/A" : "Logs")
-      ._("Diagnostics:", container.getDiagnosticsInfo());
+      ._("Diagnostics:", container.getDiagnosticsInfo() == null ?
+          "" : container.getDiagnosticsInfo());
 
     html._(InfoBlock.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
new file mode 100644
index 0000000..384a976
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
@@ -0,0 +1,86 @@
+/**
+ * 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.webapp.view.JQueryUI.tableInit;
+
+
+public class WebPageUtils {
+
+  public static String appsTableInit() {
+    return appsTableInit(false);
+  }
+
+  public static String appsTableInit(boolean isFairSchedulerPage) {
+    // id, user, name, queue, starttime, finishtime, state, status, progress, ui
+    // FairSchedulerPage's table is a bit different
+    return tableInit()
+      .append(", 'aaData': appsTableData")
+      .append(", bDeferRender: true")
+      .append(", bProcessing: true")
+      .append("\n, aoColumnDefs: ")
+      .append(getAppsTableColumnDefs(isFairSchedulerPage))
+      // Sort by id upon page load
+      .append(", aaSorting: [[0, 'desc']]}").toString();
+  }
+
+  private static String getAppsTableColumnDefs(boolean isFairSchedulerPage) {
+    StringBuilder sb = new StringBuilder();
+    return sb
+      .append("[\n")
+      .append("{'sType':'numeric', 'aTargets': [0]")
+      .append(", 'mRender': parseHadoopID }")
+      .append("\n, {'sType':'numeric', 'aTargets': " +
+          (isFairSchedulerPage ? "[6, 7]": "[5, 6]"))
+      .append(", 'mRender': renderHadoopDate }")
+      .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [9]")
+      .append(", 'mRender': parseHadoopProgress }]").toString();
+  }
+
+  public static 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();
+  }
+
+  private static 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();
+  }
+
+  public static 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();
+  }
+
+  private static String getContainersTableColumnDefs() {
+    StringBuilder sb = new StringBuilder();
+    return sb.append("[\n").append("{'sType':'numeric', 'aTargets': [0]")
+      .append(", 'mRender': parseHadoopID }]").toString();
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
index 6d94737..909bf1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebServices.java
@@ -40,7 +40,13 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppInfo;
@@ -54,10 +60,10 @@ import org.apache.hadoop.yarn.webapp.NotFoundException;
 
 public class WebServices {
 
-  protected ApplicationContext appContext;
+  protected ApplicationBaseProtocol appBaseProt;
 
-  public WebServices(ApplicationContext appContext) {
-    this.appContext = appContext;
+  public WebServices(ApplicationBaseProtocol appBaseProt) {
+    this.appBaseProt = appBaseProt;
   }
 
   public AppsInfo getApps(HttpServletRequest req, HttpServletResponse res,
@@ -144,13 +150,17 @@ public class WebServices {
     Collection<ApplicationReport> appReports = null;
     try {
       if (callerUGI == null) {
-        appReports = appContext.getAllApplications().values();
+        // TODO: the request should take the params like what RMWebServices does
+        // in YARN-1819.
+        GetApplicationsRequest request = GetApplicationsRequest.newInstance();
+        appReports = appBaseProt.getApplications(request).getApplicationList();
       } else {
         appReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationReport>> () {
           @Override
           public Collection<ApplicationReport> run() throws Exception {
-            return appContext.getAllApplications().values();
+            return appBaseProt.getApplications(
+                GetApplicationsRequest.newInstance()).getApplicationList();
           }
         });
       }
@@ -214,13 +224,17 @@ public class WebServices {
     ApplicationReport app = null;
     try {
       if (callerUGI == null) {
-        app = appContext.getApplication(id);
+        GetApplicationReportRequest request =
+            GetApplicationReportRequest.newInstance(id);
+        app = appBaseProt.getApplicationReport(request).getApplicationReport();
       } else {
         app = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationReport> () {
           @Override
           public ApplicationReport run() throws Exception {
-            return appContext.getApplication(id);
+            GetApplicationReportRequest request =
+                GetApplicationReportRequest.newInstance(id);
+            return appBaseProt.getApplicationReport(request).getApplicationReport();
           }
         });
       }
@@ -240,13 +254,20 @@ public class WebServices {
     Collection<ApplicationAttemptReport> appAttemptReports = null;
     try {
       if (callerUGI == null) {
-        appAttemptReports = appContext.getApplicationAttempts(id).values();
+        GetApplicationAttemptsRequest request =
+            GetApplicationAttemptsRequest.newInstance(id);
+        appAttemptReports =
+            appBaseProt.getApplicationAttempts(request)
+              .getApplicationAttemptList();
       } else {
         appAttemptReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ApplicationAttemptReport>> () {
           @Override
           public Collection<ApplicationAttemptReport> run() throws Exception {
-            return appContext.getApplicationAttempts(id).values();
+            GetApplicationAttemptsRequest request =
+                GetApplicationAttemptsRequest.newInstance(id);
+            return appBaseProt.getApplicationAttempts(request)
+                  .getApplicationAttemptList();
           }
         });
       }
@@ -271,13 +292,20 @@ public class WebServices {
     ApplicationAttemptReport appAttempt = null;
     try {
       if (callerUGI == null) {
-        appAttempt = appContext.getApplicationAttempt(aaid);
+        GetApplicationAttemptReportRequest request =
+            GetApplicationAttemptReportRequest.newInstance(aaid);
+        appAttempt =
+            appBaseProt.getApplicationAttemptReport(request)
+              .getApplicationAttemptReport();
       } else {
         appAttempt = callerUGI.doAs(
             new PrivilegedExceptionAction<ApplicationAttemptReport> () {
           @Override
           public ApplicationAttemptReport run() throws Exception {
-            return appContext.getApplicationAttempt(aaid);
+            GetApplicationAttemptReportRequest request =
+                GetApplicationAttemptReportRequest.newInstance(aaid);
+            return appBaseProt.getApplicationAttemptReport(request)
+                  .getApplicationAttemptReport();
           }
         });
       }
@@ -300,13 +328,16 @@ public class WebServices {
     Collection<ContainerReport> containerReports = null;
     try {
       if (callerUGI == null) {
-        containerReports = appContext.getContainers(aaid).values();
+        GetContainersRequest request = GetContainersRequest.newInstance(aaid);
+        containerReports =
+            appBaseProt.getContainers(request).getContainerList();
       } else {
         containerReports = callerUGI.doAs(
             new PrivilegedExceptionAction<Collection<ContainerReport>> () {
           @Override
           public Collection<ContainerReport> run() throws Exception {
-            return appContext.getContainers(aaid).values();
+            GetContainersRequest request = GetContainersRequest.newInstance(aaid);
+            return appBaseProt.getContainers(request).getContainerList();
           }
         });
       }
@@ -332,13 +363,18 @@ public class WebServices {
     ContainerReport container = null;
     try {
       if (callerUGI == null) {
-        container = appContext.getContainer(cid);
+        GetContainerReportRequest request =
+            GetContainerReportRequest.newInstance(cid);
+        container =
+            appBaseProt.getContainerReport(request).getContainerReport();
       } else {
         container = callerUGI.doAs(
             new PrivilegedExceptionAction<ContainerReport> () {
           @Override
           public ContainerReport run() throws Exception {
-            return appContext.getContainer(cid);
+            GetContainerReportRequest request =
+                GetContainerReportRequest.newInstance(cid);
+            return appBaseProt.getContainerReport(request).getContainerReport();
           }
         });
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
index d78f928..e8b1acc 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/dao/AppInfo.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.server.webapp.dao;
 
+import static org.apache.hadoop.yarn.util.StringHelper.CSV_JOINER;
+
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
 import javax.xml.bind.annotation.XmlRootElement;
@@ -49,6 +51,7 @@ public class AppInfo {
   protected long startedTime;
   protected long finishedTime;
   protected long elapsedTime;
+  protected String applicationTags;
 
   public AppInfo() {
     // JAXB needs this
@@ -74,7 +77,10 @@ public class AppInfo {
     finishedTime = app.getFinishTime();
     elapsedTime = Times.elapsed(startedTime, finishedTime);
     finalAppStatus = app.getFinalApplicationStatus();
-    progress = app.getProgress();
+    progress = app.getProgress() * 100; // in percent
+    if (app.getApplicationTags() != null && !app.getApplicationTags().isEmpty()) {
+      this.applicationTags = CSV_JOINER.join(app.getApplicationTags());
+    }
   }
 
   public String getAppId() {
@@ -149,4 +155,7 @@ public class AppInfo {
     return elapsedTime;
   }
 
+  public String getApplicationTags() {
+    return applicationTags;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
new file mode 100644
index 0000000..92eae48
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
@@ -0,0 +1,55 @@
+/**
+ * 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.resourcemanager.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 org.apache.hadoop.yarn.server.webapp.AppAttemptBlock;
+import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
+import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
+
+
+public class AppAttemptPage extends RmView {
+
+  @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"), WebPageUtils.containersTableInit());
+    setTableStyles(html, "containers", ".queue {width:6em}", ".ui {width:8em}");
+  }
+
+  @Override
+  protected Class<? extends SubView> content() {
+    return AppAttemptBlock.class;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
deleted file mode 100644
index 00508b8..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
+++ /dev/null
@@ -1,344 +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.resourcemanager.webapp;
-
-import static org.apache.hadoop.yarn.util.StringHelper.join;
-import static org.apache.hadoop.yarn.webapp.YarnWebParams.APPLICATION_ID;
-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._TH;
-
-import java.util.Collection;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.QueueACL;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptMetrics;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppAttemptInfo;
-import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo;
-import org.apache.hadoop.yarn.util.Apps;
-import org.apache.hadoop.yarn.util.Times;
-import org.apache.hadoop.yarn.util.resource.Resources;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TABLE;
-import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.TBODY;
-import org.apache.hadoop.yarn.webapp.util.WebAppUtils;
-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 {
-
-  private final Configuration conf;
-  private final ResourceManager rm;
-  private final boolean rmWebAppUIActions;
-
-  @Inject
-  AppBlock(ResourceManager rm, ViewContext ctx, Configuration conf) {
-    super(ctx);
-    this.conf = conf;
-    this.rm = rm;
-    this.rmWebAppUIActions =
-        conf.getBoolean(YarnConfiguration.RM_WEBAPP_UI_ACTIONS_ENABLED,
-                YarnConfiguration.DEFAULT_RM_WEBAPP_UI_ACTIONS_ENABLED);
-  }
-
-  @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;
-    }
-
-    RMContext context = this.rm.getRMContext();
-    RMApp rmApp = context.getRMApps().get(appID);
-    if (rmApp == null) {
-      puts("Application not found: "+ aid);
-      return;
-    }
-    AppInfo app =
-        new AppInfo(rm, rmApp, true, WebAppUtils.getHttpSchemePrefix(conf));
-
-    // Check for the authorization.
-    String remoteUser = request().getRemoteUser();
-    UserGroupInformation callerUGI = null;
-    if (remoteUser != null) {
-      callerUGI = UserGroupInformation.createRemoteUser(remoteUser);
-    }
-    if (callerUGI != null
-        && !(this.rm.getApplicationACLsManager().checkAccess(callerUGI,
-            ApplicationAccessType.VIEW_APP, app.getUser(), appID) || this.rm
-            .getQueueACLsManager().checkAccess(callerUGI,
-                QueueACL.ADMINISTER_QUEUE, app.getQueue()))) {
-      puts("You (User " + remoteUser
-          + ") are not authorized to view application " + appID);
-      return;
-    }
-
-    setTitle(join("Application ", aid));
-
-    if (rmWebAppUIActions) {
-      // Application Kill
-      html.div()
-        .button()
-          .$onclick("confirmAction()").b("Kill Application")._()
-          ._();
-
-      StringBuilder script = new StringBuilder();
-      script.append("function confirmAction() {")
-          .append(" b = confirm(\"Are you sure?\");")
-          .append(" if (b == true) {")
-          .append(" $.ajax({")
-          .append(" type: 'PUT',")
-          .append(" url: '/ws/v1/cluster/apps/").append(aid).append("/state',")
-          .append(" contentType: 'application/json',")
-          .append(" data: '{\"state\":\"KILLED\"}',")
-          .append(" dataType: 'json'")
-          .append(" }).done(function(data){")
-          .append(" setTimeout(function(){")
-          .append(" location.href = '/cluster/app/").append(aid).append("';")
-          .append(" }, 1000);")
-          .append(" }).fail(function(data){")
-          .append(" console.log(data);")
-          .append(" });")
-          .append(" }")
-          .append("}");
-
-      html.script().$type("text/javascript")._(script.toString())._();
-    }
-
-    RMAppMetrics appMerics = rmApp.getRMAppMetrics();
-    
-    // Get attempt metrics and fields, it is possible currentAttempt of RMApp is
-    // null. In that case, we will assume resource preempted and number of Non
-    // AM container preempted on that attempt is 0
-    RMAppAttemptMetrics attemptMetrics;
-    if (null == rmApp.getCurrentAppAttempt()) {
-      attemptMetrics = null;
-    } else {
-      attemptMetrics = rmApp.getCurrentAppAttempt().getRMAppAttemptMetrics();
-    }
-    Resource attemptResourcePreempted =
-        attemptMetrics == null ? Resources.none() : attemptMetrics
-            .getResourcePreempted();
-    int attemptNumNonAMContainerPreempted =
-        attemptMetrics == null ? 0 : attemptMetrics
-            .getNumNonAMContainersPreempted();
-    
-    info("Application Overview")
-        ._("User:", app.getUser())
-        ._("Name:", app.getName())
-        ._("Application Type:", app.getApplicationType())
-        ._("Application Tags:", app.getApplicationTags())
-        ._("YarnApplicationState:", clarifyAppState(app.getState()))
-        ._("FinalStatus Reported by AM:",
-          clairfyAppFinalStatus(app.getFinalStatus()))
-        ._("Started:", Times.format(app.getStartTime()))
-        ._("Elapsed:",
-            StringUtils.formatTime(Times.elapsed(app.getStartTime(),
-                app.getFinishTime())))
-        ._("Tracking URL:",
-            !app.isTrackingUrlReady() ? "#" : app.getTrackingUrlPretty(),
-            app.getTrackingUI())
-        ._("Diagnostics:", app.getNote());
-
-    DIV<Hamlet> pdiv = html.
-        _(InfoBlock.class).
-        div(_INFO_WRAP);
-    info("Application Overview").clear();
-    info("Application Metrics")
-        ._("Total Resource Preempted:",
-          appMerics.getResourcePreempted())
-        ._("Total Number of Non-AM Containers Preempted:",
-          String.valueOf(appMerics.getNumNonAMContainersPreempted()))
-        ._("Total Number of AM Containers Preempted:",
-          String.valueOf(appMerics.getNumAMContainersPreempted()))
-        ._("Resource Preempted from Current Attempt:",
-          attemptResourcePreempted)
-        ._("Number of Non-AM Containers Preempted from Current Attempt:",
-          attemptNumNonAMContainerPreempted)
-        ._("Aggregate Resource Allocation:",
-          String.format("%d MB-seconds, %d vcore-seconds", 
-              appMerics.getMemorySeconds(), appMerics.getVcoreSeconds()));
-    pdiv._();
-
-    Collection<RMAppAttempt> attempts = rmApp.getAppAttempts().values();
-    String amString =
-        attempts.size() == 1 ? "ApplicationMaster" : "ApplicationMasters";
-
-    DIV<Hamlet> div = html.
-        _(InfoBlock.class).
-        div(_INFO_WRAP);
-    // MRAppMasters Table
-    TABLE<DIV<Hamlet>> table = div.table("#app");
-    table.
-      tr().
-        th(amString).
-      _().
-      tr().
-        th(_TH, "Attempt Number").
-        th(_TH, "Start Time").
-        th(_TH, "Node").
-        th(_TH, "Logs").
-      _();
-
-    boolean odd = false;
-    for (RMAppAttempt attempt : attempts) {
-      AppAttemptInfo attemptInfo = new AppAttemptInfo(attempt, app.getUser());
-      table.tr((odd = !odd) ? _ODD : _EVEN).
-        td(String.valueOf(attemptInfo.getAttemptId())).
-        td(Times.format(attemptInfo.getStartTime())).
-        td().a(".nodelink", url("//",
-            attemptInfo.getNodeHttpAddress()),
-            attemptInfo.getNodeHttpAddress())._().
-        td().a(".logslink", url(attemptInfo.getLogsLink()), "logs")._().
-      _();
-    }
-
-    table._();
-    div._();
-
-    createContainerLocalityTable(html, attemptMetrics);
-    createResourceRequestsTable(html, app);
-  }
-
-  private void createContainerLocalityTable(Block html,
-      RMAppAttemptMetrics attemptMetrics) {
-    if (attemptMetrics == null) {
-      return;
-    }
-
-    DIV<Hamlet> div = html.div(_INFO_WRAP);
-    TABLE<DIV<Hamlet>> table =
-        div.h3(
-          "Total Allocated Containers: "
-              + attemptMetrics.getTotalAllocatedContainers()).h3("Each table cell"
-            + " represents the number of NodeLocal/RackLocal/OffSwitch containers"
-            + " satisfied by NodeLocal/RackLocal/OffSwitch resource requests.").table(
-          "#containerLocality");
-    table.
-      tr().
-        th(_TH, "").
-        th(_TH, "Node Local Request").
-        th(_TH, "Rack Local Request").
-        th(_TH, "Off Switch Request").
-      _();
-
-    String[] containersType =
-        { "Num Node Local Containers (satisfied by)", "Num Rack Local Containers (satisfied by)",
-            "Num Off Switch Containers (satisfied by)" };
-    boolean odd = false;
-    for (int i = 0; i < attemptMetrics.getLocalityStatistics().length; i++) {
-      table.tr((odd = !odd) ? _ODD : _EVEN).td(containersType[i])
-        .td(String.valueOf(attemptMetrics.getLocalityStatistics()[i][0]))
-        .td(i == 0 ? "" : String.valueOf(attemptMetrics.getLocalityStatistics()[i][1]))
-        .td(i <= 1 ? "" : String.valueOf(attemptMetrics.getLocalityStatistics()[i][2]))._();
-    }
-    table._();
-    div._();
-  }
-
-  private void createResourceRequestsTable(Block html, AppInfo app) {
-    TBODY<TABLE<Hamlet>> tbody =
-        html.table("#ResourceRequests").thead().tr()
-          .th(".priority", "Priority")
-          .th(".resourceName", "Resource Name")
-          .th(".totalResource", "Capability")
-          .th(".numContainers", "Num Containers")
-          .th(".relaxLocality", "Relax Locality")
-          .th(".nodeLabelExpression", "Node Label Expression")._()._().tbody();
-
-    Resource totalResource = Resource.newInstance(0, 0);
-    if (app.getResourceRequests() != null) {
-      for (ResourceRequest request : app.getResourceRequests()) {
-        if (request.getNumContainers() == 0) {
-          continue;
-        }
-
-        tbody.tr()
-          .td(String.valueOf(request.getPriority()))
-          .td(request.getResourceName())
-          .td(String.valueOf(request.getCapability()))
-          .td(String.valueOf(request.getNumContainers()))
-          .td(String.valueOf(request.getRelaxLocality()))
-          .td(request.getNodeLabelExpression() == null ? "N/A" : request
-              .getNodeLabelExpression())._();
-        if (request.getResourceName().equals(ResourceRequest.ANY)) {
-          Resources.addTo(totalResource,
-            Resources.multiply(request.getCapability(),
-              request.getNumContainers()));
-        }
-      }
-    }
-    html.div().$class("totalResourceRequests")
-      .h3("Total Outstanding Resource Requests: " + totalResource)._();
-    tbody._()._();
-  }
-
-  private String clarifyAppState(YarnApplicationState state) {
-    String ret = state.toString();
-    switch (state) {
-    case NEW:
-      return ret + ": waiting for application to be initialized";
-    case NEW_SAVING:
-      return ret + ": waiting for application to be persisted in state-store.";
-    case SUBMITTED:
-      return ret + ": waiting for application to be accepted by scheduler.";
-    case ACCEPTED:
-      return ret + ": waiting for AM container to be allocated, launched and"
-          + " register with RM.";
-    case RUNNING:
-      return ret + ": AM has registered with RM and started running.";
-    default:
-      return ret;
-    }
-  }
-
-  private String clairfyAppFinalStatus(FinalApplicationStatus status) {
-    if (status == FinalApplicationStatus.UNDEFINED) {
-      return "Application has not completed yet.";
-    }
-    return status.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/95bfd087/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
index 8993324..9f9b7c9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppPage.java
@@ -18,19 +18,38 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.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 org.apache.hadoop.yarn.server.webapp.AppBlock;
+import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
+import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 public class AppPage extends RmView {
 
-  @Override protected void preHead(Page.HTML<_> html) {
+  @Override 
+  protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
-    set(DATATABLES_ID, "ResourceRequests");
+    String appId = $(YarnWebParams.APPLICATION_ID);
+    set(
+      TITLE,
+      appId.isEmpty() ? "Bad request: missing application ID" : join(
+        "Application ", $(YarnWebParams.APPLICATION_ID)));
+
+    set(DATATABLES_ID, "attempts ResourceRequests");
+    set(initID(DATATABLES, "attempts"), WebPageUtils.attemptsTableInit());
+    setTableStyles(html, "attempts", ".queue {width:6em}", ".ui {width:8em}");
+
     setTableStyles(html, "ResourceRequests");
+
+    set(YarnWebParams.WEB_UI_TYPE, YarnWebParams.RM_WEB_UI);
   }
 
-  @Override protected Class<? extends SubView> content() {
+  @Override 
+  protected Class<? extends SubView> content() {
     return AppBlock.class;
   }
 }