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 vi...@apache.org on 2015/09/02 00:20:50 UTC

[1/4] hadoop git commit: YARN-1809. Synchronize RM and TimeLineServer Web-UIs. Contributed by Zhijie Shen and Xuan Gong (cherry picked from commit 95bfd087dc89e57a93340604cc8b96042fa1a05a)

Repository: hadoop
Updated Branches:
  refs/heads/branch-2.6.1 81417f757 -> b25491dc4


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.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/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 89b4a78..ffead48 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerLeafQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.CapacitySchedulerQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ResourceInfo;
+import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.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/ContainerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/ContainerPage.java
new file mode 100644
index 0000000..b8cd1ad
--- /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/ContainerPage.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.server.resourcemanager.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 RmView {
+
+  @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;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.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/DefaultSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
index e05987b..1c8828c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/DefaultSchedulerPage.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.yarn.util.StringHelper.join;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FifoSchedulerInfo;
+import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet.DIV;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.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/FairSchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
index 0d3c297..2902cc7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/FairSchedulerPage.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairSchedule
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerLeafQueueInfo;
 import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.FairSchedulerQueueInfo;
+import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.ResponseInfo;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
@@ -221,6 +222,11 @@ public class FairSchedulerPage extends RmView {
     return QueuesBlock.class;
   }
 
+  @Override
+  protected String initAppsTable() {
+    return WebPageUtils.appsTableInit(true);
+  }
+
   static String percent(float f) {
     return String.format("%.1f%%", f * 100);
   }
@@ -232,19 +238,4 @@ public class FairSchedulerPage extends RmView {
   static String left(float f) {
     return String.format("left:%.1f%%", f * 100);
   }
-  
-  @Override
-  protected String getAppsTableColumnDefs() {
-    StringBuilder sb = new StringBuilder();
-    return sb
-      .append("[\n")
-      .append("{'sType':'numeric', 'aTargets': [0]")
-      .append(", 'mRender': parseHadoopID }")
-
-      .append("\n, {'sType':'numeric', 'aTargets': [6, 7]")
-      .append(", 'mRender': renderHadoopDate }")
-
-      .append("\n, {'sType':'numeric', bSearchable:false, 'aTargets': [9]")
-      .append(", 'mRender': parseHadoopProgress }]").toString();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.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/RMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
index 67c73b8..066ba52 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebApp.java
@@ -23,9 +23,10 @@ import static org.apache.hadoop.yarn.util.StringHelper.pajoin;
 import java.net.InetSocketAddress;
 
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.util.RMHAUtils;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
+import org.apache.hadoop.yarn.util.RMHAUtils;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
@@ -53,6 +54,7 @@ public class RMWebApp extends WebApp implements YarnWebParams {
 
     if (rm != null) {
       bind(ResourceManager.class).toInstance(rm);
+      bind(ApplicationBaseProtocol.class).toInstance(rm.getClientRMService());
     }
     route("/", RmController.class);
     route(pajoin("/nodes", NODE_STATE), RmController.class, "nodes");
@@ -61,6 +63,9 @@ public class RMWebApp extends WebApp implements YarnWebParams {
     route(pajoin("/app", APPLICATION_ID), RmController.class, "app");
     route("/scheduler", RmController.class, "scheduler");
     route(pajoin("/queue", QUEUE_NAME), RmController.class, "queue");
+    route(pajoin("/appattempt", APPLICATION_ATTEMPT_ID), RmController.class,
+      "appattempt");
+    route(pajoin("/container", CONTAINER_ID), RmController.class, "container");
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.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/RmController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
index f186bf4..e65c57d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmController.java
@@ -55,6 +55,14 @@ public class RmController extends Controller {
     render(AppPage.class);
   }
 
+  public void appattempt() {
+    render(AppAttemptPage.class);
+  }
+
+  public void container() {
+    render(ContainerPage.class);
+  }
+
   public void nodes() {
     render(NodesPage.class);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.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/RmView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
index 769c4da..1a437f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RmView.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import org.apache.hadoop.yarn.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
 
@@ -35,7 +36,7 @@ public class RmView extends TwoColumnLayout {
   protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "apps");
-    set(initID(DATATABLES, "apps"), appsTableInit());
+    set(initID(DATATABLES, "apps"), initAppsTable());
     setTableStyles(html, "apps", ".queue {width:6em}", ".ui {width:8em}");
 
     // Set the correct title.
@@ -59,31 +60,7 @@ public class RmView extends TwoColumnLayout {
     return AppsBlockWithMetrics.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':'string', '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();
+  protected String initAppsTable() {
+    return WebPageUtils.appsTableInit();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
index 9732c19..8c7b14d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestAppPage.java
@@ -23,6 +23,7 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Resource;
@@ -32,6 +33,7 @@ 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.RMAppState;
+import org.apache.hadoop.yarn.server.webapp.AppBlock;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
 import org.junit.Test;
@@ -75,8 +77,10 @@ public class TestAppPage {
               @Override
               public void configure(Binder binder) {
                 try {
-                  binder.bind(ResourceManager.class).toInstance(
-                      TestRMWebApp.mockRm(rmContext));
+                  ResourceManager rm = TestRMWebApp.mockRm(rmContext);
+                  binder.bind(ResourceManager.class).toInstance(rm);
+                  binder.bind(ApplicationBaseProtocol.class).toInstance(
+                    rm.getClientRMService());
                 } catch (IOException e) {
                   throw new IllegalStateException(e);
                 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
index abc701d..f640cc8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebApp.java
@@ -21,19 +21,30 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 import static org.apache.hadoop.yarn.server.resourcemanager.MockNodes.newResource;
 import static org.apache.hadoop.yarn.webapp.Params.TITLE;
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 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.NodeId;
 import org.apache.hadoop.yarn.api.records.NodeState;
+import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNodes;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
@@ -54,6 +65,7 @@ import org.apache.hadoop.yarn.util.StringHelper;
 import org.apache.hadoop.yarn.webapp.WebApps;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 import org.apache.hadoop.yarn.webapp.test.WebAppTests;
+import org.junit.Assert;
 import org.junit.Test;
 
 import com.google.common.collect.Maps;
@@ -87,7 +99,10 @@ public class TestRMWebApp {
       @Override
       public void configure(Binder binder) {
         try {
-          binder.bind(ResourceManager.class).toInstance(mockRm(3, 1, 2, 8*GiB));
+          ResourceManager mockRm = mockRm(3, 1, 2, 8*GiB);
+          binder.bind(ResourceManager.class).toInstance(mockRm);
+          binder.bind(ApplicationBaseProtocol.class)
+              .toInstance(mockRm.getClientRMService());
         } catch (IOException e) {
           throw new IllegalStateException(e);
         }
@@ -194,9 +209,11 @@ public class TestRMWebApp {
     ResourceManager rm = mock(ResourceManager.class);
     ResourceScheduler rs = mockCapacityScheduler();
     ApplicationACLsManager aclMgr = mockAppACLsManager();
+    ClientRMService clientRMService = mockClientRMService(rmContext);
     when(rm.getResourceScheduler()).thenReturn(rs);
     when(rm.getRMContext()).thenReturn(rmContext);
     when(rm.getApplicationACLsManager()).thenReturn(aclMgr);
+    when(rm.getClientRMService()).thenReturn(clientRMService);
     return rm;
   }
 
@@ -222,6 +239,35 @@ public class TestRMWebApp {
     return new ApplicationACLsManager(conf);
   }
 
+  public static ClientRMService mockClientRMService(RMContext rmContext) {
+    ClientRMService clientRMService = mock(ClientRMService.class);
+    List<ApplicationReport> appReports = new ArrayList<ApplicationReport>();
+    for (RMApp app : rmContext.getRMApps().values()) {
+      ApplicationReport appReport =
+          ApplicationReport.newInstance(
+              app.getApplicationId(), (ApplicationAttemptId) null,
+              app.getUser(), app.getQueue(),
+              app.getName(), (String) null, 0, (Token) null,
+              app.createApplicationState(),
+              app.getDiagnostics().toString(), (String) null,
+              app.getStartTime(), app.getFinishTime(),
+              app.getFinalApplicationStatus(),
+              (ApplicationResourceUsageReport) null, app.getTrackingUrl(),
+              app.getProgress(), app.getApplicationType(), (Token) null);
+      appReports.add(appReport);
+    }
+    GetApplicationsResponse response = mock(GetApplicationsResponse.class);
+    when(response.getApplicationList()).thenReturn(appReports);
+    try {
+      when(clientRMService.getApplications(any(GetApplicationsRequest.class)))
+          .thenReturn(response);
+    } catch (YarnException e) {
+      Assert.fail("Exception is not expteced.");
+    }
+    return clientRMService;
+  }
+
+
   static void setupQueueConfiguration(CapacitySchedulerConfiguration conf) {
     // Define top-level queues
     conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
index f07cb8d..f03f2c4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebAppFairScheduler.java
@@ -23,10 +23,12 @@ import com.google.inject.Binder;
 import com.google.inject.Injector;
 import com.google.inject.Module;
 import org.apache.hadoop.util.StringUtils;
+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.Resource;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
@@ -72,7 +74,8 @@ public class TestRMWebAppFairScheduler {
                   mockRm(rmContext);
               binder.bind(ResourceManager.class).toInstance
                   (mockRmWithFairScheduler);
-
+              binder.bind(ApplicationBaseProtocol.class).toInstance(
+                mockRmWithFairScheduler.getClientRMService());
             } catch (IOException e) {
               throw new IllegalStateException(e);
             }
@@ -111,6 +114,8 @@ public class TestRMWebAppFairScheduler {
                   mockRmWithApps(rmContext);
               binder.bind(ResourceManager.class).toInstance
                   (mockRmWithFairScheduler);
+              binder.bind(ApplicationBaseProtocol.class).toInstance(
+                  mockRmWithFairScheduler.getClientRMService());
 
             } catch (IOException e) {
               throw new IllegalStateException(e);
@@ -162,8 +167,10 @@ public class TestRMWebAppFairScheduler {
       IOException {
     ResourceManager rm = mock(ResourceManager.class);
     ResourceScheduler rs = mockFairScheduler();
+    ClientRMService clientRMService = mockClientRMService(rmContext);
     when(rm.getResourceScheduler()).thenReturn(rs);
     when(rm.getRMContext()).thenReturn(rmContext);
+    when(rm.getClientRMService()).thenReturn(clientRMService);
     return rm;
   }
 
@@ -182,8 +189,10 @@ public class TestRMWebAppFairScheduler {
       IOException {
     ResourceManager rm = mock(ResourceManager.class);
     ResourceScheduler rs =  mockFairSchedulerWithoutApps(rmContext);
+    ClientRMService clientRMService = mockClientRMService(rmContext);
     when(rm.getResourceScheduler()).thenReturn(rs);
     when(rm.getRMContext()).thenReturn(rmContext);
+    when(rm.getClientRMService()).thenReturn(clientRMService);
     return rm;
   }
 
@@ -207,4 +216,7 @@ public class TestRMWebAppFairScheduler {
     return fs;
   }
 
+  public static ClientRMService mockClientRMService(RMContext rmContext) {
+    return mock(ClientRMService.class);
+  }
 }


[2/4] hadoop git commit: YARN-1809. Synchronize RM and TimeLineServer Web-UIs. Contributed by Zhijie Shen and Xuan Gong (cherry picked from commit 95bfd087dc89e57a93340604cc8b96042fa1a05a)

Posted by vi...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/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/7ffdf7d1/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 76bf8c3..ebb2ec8 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 JerseyTest {
 
-  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 JerseyTest {
     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 JerseyTest {
       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 JerseyTest {
     assertEquals(ContainerState.COMPLETE.toString(),
       container.getString("containerState"));
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/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/7ffdf7d1/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/7ffdf7d1/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..37b60bb 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) == ',') {
@@ -212,4 +327,30 @@ public class AppBlock extends HtmlBlock {
 
     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/7ffdf7d1/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/7ffdf7d1/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/7ffdf7d1/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/7ffdf7d1/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 385d10a..a02b80f 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
@@ -39,7 +39,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;
@@ -53,10 +59,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,
@@ -143,13 +149,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();
           }
         });
       }
@@ -213,13 +223,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();
           }
         });
       }
@@ -239,13 +253,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();
           }
         });
       }
@@ -270,13 +291,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();
           }
         });
       }
@@ -299,13 +327,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();
           }
         });
       }
@@ -331,13 +362,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/7ffdf7d1/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/7ffdf7d1/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/7ffdf7d1/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 bae4947..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,265 +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.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.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(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._();
-  }
-
-  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/7ffdf7d1/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 a55c62f..77ae44c 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,15 +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);
+
+    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;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.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/AppsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
deleted file mode 100644
index 054a1a7..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
+++ /dev/null
@@ -1,131 +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.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.util.Collection;
-import java.util.HashSet;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.commons.lang.StringEscapeUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
-import org.apache.hadoop.yarn.server.resourcemanager.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.util.WebAppUtils;
-import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
-
-import com.google.inject.Inject;
-
-class AppsBlock extends HtmlBlock {
-  final ConcurrentMap<ApplicationId, RMApp> apps;
-  private final Configuration conf;
-
-  @Inject
-  AppsBlock(ResourceManager rm, ViewContext ctx, Configuration conf) {
-    super(ctx);
-    apps = rm.getRMContext().getRMApps();
-    this.conf = conf;
-  }
-
-  @Override public void render(Block html) {
-    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", "YarnApplicationState").
-            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));
-      }
-    }
-    StringBuilder appsTableData = new StringBuilder("[\n");
-    for (RMApp app : apps.values()) {
-      if (reqAppStates != null && !reqAppStates.contains(app.createApplicationState())) {
-        continue;
-      }
-      AppInfo appInfo = new AppInfo(app, true, WebAppUtils.getHttpSchemePrefix(conf));
-      String percent = String.format("%.1f", appInfo.getProgress());
-      //AppID numerical value parsed by parseHadoopID in yarn.dt.plugins.js
-      appsTableData.append("[\"<a href='")
-      .append(url("app", appInfo.getAppId())).append("'>")
-      .append(appInfo.getAppId()).append("</a>\",\"")
-      .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
-        appInfo.getUser()))).append("\",\"")
-      .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
-        appInfo.getName()))).append("\",\"")
-      .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
-        appInfo.getApplicationType()))).append("\",\"")
-      .append(StringEscapeUtils.escapeJavaScript(StringEscapeUtils.escapeHtml(
-        appInfo.getQueue()))).append("\",\"")
-      .append(appInfo.getStartTime()).append("\",\"")
-      .append(appInfo.getFinishTime()).append("\",\"")
-      .append(appInfo.getState()).append("\",\"")
-      .append(appInfo.getFinalStatus() == FinalApplicationStatus.UNDEFINED ?
-          "N/A" : appInfo.getFinalStatus()).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 =
-        !appInfo.isTrackingUrlReady()? "#" : appInfo.getTrackingUrlPretty();
-      
-      appsTableData.append(trackingURL).append("'>")
-      .append(appInfo.getTrackingUI()).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._()._();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.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/AppsBlockWithMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
index 6d461f6..cb0836a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlockWithMetrics.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
+import org.apache.hadoop.yarn.server.webapp.AppsBlock;
 import org.apache.hadoop.yarn.webapp.view.HtmlBlock;
 
 /**


[3/4] hadoop git commit: YARN-1809. Synchronize RM and TimeLineServer Web-UIs. Contributed by Zhijie Shen and Xuan Gong (cherry picked from commit 95bfd087dc89e57a93340604cc8b96042fa1a05a)

Posted by vi...@apache.org.
YARN-1809. Synchronize RM and TimeLineServer Web-UIs. Contributed by Zhijie Shen and Xuan Gong
(cherry picked from commit 95bfd087dc89e57a93340604cc8b96042fa1a05a)

(cherry picked from commit a5f3fb4dc14503bf7c454a48cf954fb0d6710de2)
(cherry picked from commit 27a2f0acb84202cc082090eef7eea57f6e42f9bb)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/7ffdf7d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7ffdf7d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7ffdf7d1

Branch: refs/heads/branch-2.6.1
Commit: 7ffdf7d105fd2ce7a484cb96a96f414670bec141
Parents: 81417f7
Author: Jian He <ji...@apache.org>
Authored: Thu Mar 5 21:14:41 2015 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Sep 1 15:12:53 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/api/ApplicationBaseProtocol.java       | 355 +++++++++++++++++++
 .../yarn/api/ApplicationClientProtocol.java     | 290 +--------------
 .../yarn/api/ApplicationHistoryProtocol.java    | 303 +---------------
 .../apache/hadoop/yarn/webapp/ResponseInfo.java |   6 +-
 .../hadoop/yarn/webapp/YarnWebParams.java       |   4 +
 .../hadoop/yarn/webapp/view/HtmlBlock.java      |   2 +
 .../ApplicationHistoryClientService.java        | 176 +++++----
 .../ApplicationHistoryManager.java              | 126 ++++++-
 .../ApplicationHistoryServer.java               |   2 +-
 .../webapp/AHSView.java                         |  28 +-
 .../webapp/AHSWebApp.java                       |  16 +-
 .../webapp/AHSWebServices.java                  |   6 +-
 .../webapp/AppAttemptPage.java                  |  15 +-
 .../webapp/AppPage.java                         |  21 +-
 .../TestApplicationHistoryClientService.java    |  12 +-
 .../webapp/TestAHSWebApp.java                   |  27 +-
 .../webapp/TestAHSWebServices.java              |  26 +-
 .../yarn/server/api/ApplicationContext.java     | 122 -------
 .../yarn/server/webapp/AppAttemptBlock.java     | 119 ++++---
 .../hadoop/yarn/server/webapp/AppBlock.java     | 197 ++++++++--
 .../hadoop/yarn/server/webapp/AppsBlock.java    |  53 ++-
 .../yarn/server/webapp/ContainerBlock.java      |  29 +-
 .../hadoop/yarn/server/webapp/WebPageUtils.java |  86 +++++
 .../hadoop/yarn/server/webapp/WebServices.java  |  68 +++-
 .../hadoop/yarn/server/webapp/dao/AppInfo.java  |  11 +-
 .../resourcemanager/webapp/AppAttemptPage.java  |  55 +++
 .../server/resourcemanager/webapp/AppBlock.java | 265 --------------
 .../server/resourcemanager/webapp/AppPage.java  |  27 +-
 .../resourcemanager/webapp/AppsBlock.java       | 131 -------
 .../webapp/AppsBlockWithMetrics.java            |   1 +
 .../webapp/CapacitySchedulerPage.java           |   1 +
 .../resourcemanager/webapp/ContainerPage.java   |  44 +++
 .../webapp/DefaultSchedulerPage.java            |   1 +
 .../webapp/FairSchedulerPage.java               |  21 +-
 .../server/resourcemanager/webapp/RMWebApp.java |   7 +-
 .../resourcemanager/webapp/RmController.java    |   8 +
 .../server/resourcemanager/webapp/RmView.java   |  31 +-
 .../resourcemanager/webapp/TestAppPage.java     |   8 +-
 .../resourcemanager/webapp/TestRMWebApp.java    |  48 ++-
 .../webapp/TestRMWebAppFairScheduler.java       |  14 +-
 41 files changed, 1301 insertions(+), 1464 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5317b59..b8c15f9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -13,6 +13,9 @@ Release 2.6.1 - UNRELEASED
 
     YARN-3230. Clarify application states on the web UI. (Jian He via wangda)
 
+    YARN-1809. Synchronize RM and TimeLineServer Web-UIs. (Zhijie Shen and
+    Xuan Gong via jianhe)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
new file mode 100644
index 0000000..2a8a283
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
@@ -0,0 +1,355 @@
+/**
+ * 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.api;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+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.api.records.Token;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * <p>
+ * The protocol between clients and the <code>ResourceManager</code> or
+ * <code>ApplicationHistoryServer</code> to get information on applications,
+ * application attempts and containers.
+ * </p>
+ *
+ */
+@Private
+@Unstable
+public interface ApplicationBaseProtocol {
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of an Application from the
+   * <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>.
+   * </p>
+   *
+   * <p>
+   * The client, via {@link GetApplicationReportRequest} provides the
+   * {@link ApplicationId} of the application.
+   * </p>
+   *
+   * <p>
+   * In secure mode,the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code> verifies access to the application,
+   * queue etc. before accepting the request.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetApplicationReportResponse} which includes the
+   * {@link ApplicationReport} for the application.
+   * </p>
+   *
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access then the following
+   * fields in the report will be set to stubbed values:
+   * <ul>
+   * <li>host - set to "N/A"</li>
+   * <li>RPC port - set to -1</li>
+   * <li>client token - set to "N/A"</li>
+   * <li>diagnostics - set to "N/A"</li>
+   * <li>tracking URL - set to "N/A"</li>
+   * <li>original tracking URL - set to "N/A"</li>
+   * <li>resource usage report - all values are -1</li>
+   * </ul>
+   * </p>
+   *
+   * @param request
+   *          request for an application report
+   * @return application report
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Stable
+  @Idempotent
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of Applications matching the
+   * filters defined by {@link GetApplicationsRequest} in the cluster from the
+   * <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetApplicationsResponse} which includes the
+   * {@link ApplicationReport} for the applications.
+   * </p>
+   *
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access for an application
+   * then the corresponding report will be filtered as described in
+   * {@link #getApplicationReport(GetApplicationReportRequest)}.
+   * </p>
+   *
+   * @param request
+   *          request for report on applications
+   * @return report on applications matching the given application types defined
+   *         in the request
+   * @throws YarnException
+   * @throws IOException
+   * @see GetApplicationsRequest
+   */
+  @Public
+  @Stable
+  @Idempotent
+  public GetApplicationsResponse
+      getApplications(GetApplicationsRequest request) throws YarnException,
+          IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of an Application Attempt
+   * from the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code>
+   * </p>
+   *
+   * <p>
+   * The client, via {@link GetApplicationAttemptReportRequest} provides the
+   * {@link ApplicationAttemptId} of the application attempt.
+   * </p>
+   *
+   * <p>
+   * In secure mode,the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code> verifies access to the method before
+   * accepting the request.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetApplicationAttemptReportResponse} which includes
+   * the {@link ApplicationAttemptReport} for the application attempt.
+   * </p>
+   *
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access then the following
+   * fields in the report will be set to stubbed values:
+   * <ul>
+   * <li>host</li>
+   * <li>RPC port</li>
+   * <li>client token</li>
+   * <li>diagnostics - set to "N/A"</li>
+   * <li>tracking URL</li>
+   * </ul>
+   * </p>
+   *
+   * @param request
+   *          request for an application attempt report
+   * @return application attempt report
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  @Idempotent
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request) throws YarnException,
+      IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of all Application attempts
+   * in the cluster from the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code>
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetApplicationAttemptsRequest} which includes the
+   * {@link ApplicationAttemptReport} for all the applications attempts of a
+   * specified application attempt.
+   * </p>
+   *
+   * <p>
+   * If the user does not have <code>VIEW_APP</code> access for an application
+   * then the corresponding report will be filtered as described in
+   * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
+   * </p>
+   *
+   * @param request
+   *          request for reports on all application attempts of an application
+   * @return reports on all application attempts of an application
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  @Idempotent
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of an Container from the
+   * <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * </p>
+   *
+   * <p>
+   * The client, via {@link GetContainerReportRequest} provides the
+   * {@link ContainerId} of the container.
+   * </p>
+   *
+   * <p>
+   * In secure mode,the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code> verifies access to the method before
+   * accepting the request.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetContainerReportResponse} which includes the
+   * {@link ContainerReport} for the container.
+   * </p>
+   *
+   * @param request
+   *          request for a container report
+   * @return container report
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  @Idempotent
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get a report of Containers for an
+   * application attempt from the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code>
+   * </p>
+   *
+   * <p>
+   * The client, via {@link GetContainersRequest} provides the
+   * {@link ApplicationAttemptId} of the application attempt.
+   * </p>
+   *
+   * <p>
+   * In secure mode,the <code>ResourceManager</code> or
+   * <code>ApplicationHistoryServer</code> verifies access to the method before
+   * accepting the request.
+   * </p>
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with a {@link GetContainersResponse} which includes a list of
+   * {@link ContainerReport} for all the containers of a specific application
+   * attempt.
+   * </p>
+   *
+   * @param request
+   *          request for a list of container reports of an application attempt.
+   * @return reports on all containers of an application attempt
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  @Idempotent
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException;
+
+  /**
+   * <p>
+   * The interface used by clients to get delegation token, enabling the
+   * containers to be able to talk to the service using those tokens.
+   *
+   * <p>
+   * The <code>ResourceManager</code> or <code>ApplicationHistoryServer</code>
+   * responds with the delegation {@link Token} that can be used by the client
+   * to speak to this service.
+   *
+   * @param request
+   *          request to get a delegation token for the client.
+   * @return delegation token that can be used to talk to this service
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Stable
+  @Idempotent
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException;
+
+  /**
+   * Renew an existing delegation {@link Token}.
+   *
+   * @param request
+   *          the delegation token to be renewed.
+   * @return the new expiry time for the delegation token.
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Private
+  @Unstable
+  @Idempotent
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException;
+
+  /**
+   * Cancel an existing delegation {@link Token}.
+   *
+   * @param request
+   *          the delegation token to be cancelled.
+   * @return an empty response.
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Private
+  @Unstable
+  @Idempotent
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
index add35f1..004b9aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationClientProtocol.java
@@ -20,33 +20,17 @@ package org.apache.hadoop.yarn.api;
 
 import java.io.IOException;
 
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.io.retry.Idempotent;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest;
@@ -59,8 +43,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.MoveApplicationAcrossQueuesResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationDeleteResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationSubmissionRequest;
@@ -69,19 +51,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.ReservationUpdateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse;
-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.ApplicationSubmissionContext;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
-import org.apache.hadoop.yarn.api.records.ContainerReport;
 import org.apache.hadoop.yarn.api.records.NodeReport;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
 import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
 import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException;
@@ -94,7 +70,7 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
  */
 @Public
 @Stable
-public interface ApplicationClientProtocol {
+public interface ApplicationClientProtocol extends ApplicationBaseProtocol {
   /**
    * <p>The interface used by clients to obtain a new {@link ApplicationId} for 
    * submitting new applications.</p>
@@ -197,44 +173,6 @@ public interface ApplicationClientProtocol {
   throws YarnException, IOException;
 
   /**
-   * <p>The interface used by clients to get a report of an Application from
-   * the <code>ResourceManager</code>.</p>
-   * 
-   * <p>The client, via {@link GetApplicationReportRequest} provides the
-   * {@link ApplicationId} of the application.</p>
-   *
-   * <p> In secure mode,the <code>ResourceManager</code> verifies access to the
-   * application, queue etc. before accepting the request.</p> 
-   * 
-   * <p>The <code>ResourceManager</code> responds with a 
-   * {@link GetApplicationReportResponse} which includes the 
-   * {@link ApplicationReport} for the application.</p>
-   * 
-   * <p>If the user does not have <code>VIEW_APP</code> access then the
-   * following fields in the report will be set to stubbed values:
-   * <ul>
-   *   <li>host - set to "N/A"</li>
-   *   <li>RPC port - set to -1</li>
-   *   <li>client token - set to "N/A"</li>
-   *   <li>diagnostics - set to "N/A"</li>
-   *   <li>tracking URL - set to "N/A"</li>
-   *   <li>original tracking URL - set to "N/A"</li>
-   *   <li>resource usage report - all values are -1</li>
-   * </ul></p>
-   *
-   * @param request request for an application report
-   * @return application report 
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Stable
-  @Idempotent
-  public GetApplicationReportResponse getApplicationReport(
-      GetApplicationReportRequest request) 
-  throws YarnException, IOException;
-  
-  /**
    * <p>The interface used by clients to get metrics about the cluster from
    * the <code>ResourceManager</code>.</p>
    * 
@@ -254,35 +192,7 @@ public interface ApplicationClientProtocol {
   public GetClusterMetricsResponse getClusterMetrics(
       GetClusterMetricsRequest request) 
   throws YarnException, IOException;
-  
-  /**
-   * <p>The interface used by clients to get a report of Applications
-   * matching the filters defined by {@link GetApplicationsRequest}
-   * in the cluster from the <code>ResourceManager</code>.</p>
-   * 
-   * <p>The <code>ResourceManager</code> responds with a 
-   * {@link GetApplicationsResponse} which includes the
-   * {@link ApplicationReport} for the applications.</p>
-   * 
-   * <p>If the user does not have <code>VIEW_APP</code> access for an
-   * application then the corresponding report will be filtered as
-   * described in {@link #getApplicationReport(GetApplicationReportRequest)}.
-   * </p>
-   *
-   * @param request request for report on applications
-   * @return report on applications matching the given application types
-   *           defined in the request
-   * @throws YarnException
-   * @throws IOException
-   * @see GetApplicationsRequest
-   */
-  @Public
-  @Stable
-  @Idempotent
-  public GetApplicationsResponse getApplications(
-      GetApplicationsRequest request)
-  throws YarnException, IOException;
-  
+
   /**
    * <p>The interface used by clients to get a report of all nodes
    * in the cluster from the <code>ResourceManager</code>.</p>
@@ -344,57 +254,8 @@ public interface ApplicationClientProtocol {
   public GetQueueUserAclsInfoResponse getQueueUserAcls(
       GetQueueUserAclsInfoRequest request) 
   throws YarnException, IOException;
-  
-  /**
-   * <p>The interface used by clients to get delegation token, enabling the 
-   * containers to be able to talk to the service using those tokens.
-   * 
-   *  <p> The <code>ResourceManager</code> responds with the delegation
-   *  {@link Token} that can be used by the client to speak to this
-   *  service.
-   * @param request request to get a delegation token for the client.
-   * @return delegation token that can be used to talk to this service
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Stable
-  @Idempotent
-  public GetDelegationTokenResponse getDelegationToken(
-      GetDelegationTokenRequest request) 
-  throws YarnException, IOException;
-  
-  /**
-   * Renew an existing delegation {@link Token}.
-   * 
-   * @param request the delegation token to be renewed.
-   * @return the new expiry time for the delegation token.
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Private
-  @Unstable
-  @Idempotent
-  public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnException,
-      IOException;
 
   /**
-   * Cancel an existing delegation {@link Token}.
-   * 
-   * @param request the delegation token to be cancelled.
-   * @return an empty response.
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Private
-  @Unstable
-  @Idempotent
-  public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnException,
-      IOException;
-  
-  /**
    * Move an application to a new queue.
    * 
    * @param request the application ID and the target queue
@@ -410,153 +271,6 @@ public interface ApplicationClientProtocol {
 
   /**
    * <p>
-   * The interface used by clients to get a report of an Application Attempt
-   * from the <code>ResourceManager</code> 
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetApplicationAttemptReportRequest} provides the
-   * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ResourceManager</code> verifies access to
-   * the method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ResourceManager</code> responds with a
-   * {@link GetApplicationAttemptReportResponse} which includes the
-   * {@link ApplicationAttemptReport} for the application attempt.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access then the following
-   * fields in the report will be set to stubbed values:
-   * <ul>
-   * <li>host</li>
-   * <li>RPC port</li>
-   * <li>client token</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL</li>
-   * </ul>
-   * </p>
-   * 
-   * @param request
-   *          request for an application attempt report
-   * @return application attempt report
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  @Idempotent
-  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
-      GetApplicationAttemptReportRequest request) throws YarnException,
-      IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of all Application attempts
-   * in the cluster from the <code>ResourceManager</code>
-   * </p>
-   * 
-   * <p>
-   * The <code>ResourceManager</code> responds with a
-   * {@link GetApplicationAttemptsRequest} which includes the
-   * {@link ApplicationAttemptReport} for all the applications attempts of a
-   * specified application attempt.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access for an application
-   * then the corresponding report will be filtered as described in
-   * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
-   * </p>
-   * 
-   * @param request
-   *          request for reports on all application attempts of an application
-   * @return reports on all application attempts of an application
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  @Idempotent
-  public GetApplicationAttemptsResponse getApplicationAttempts(
-      GetApplicationAttemptsRequest request) throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of an Container from the
-   * <code>ResourceManager</code>
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetContainerReportRequest} provides the
-   * {@link ContainerId} of the container.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ResourceManager</code> verifies access to the
-   * method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ResourceManager</code> responds with a
-   * {@link GetContainerReportResponse} which includes the
-   * {@link ContainerReport} for the container.
-   * </p>
-   * 
-   * @param request
-   *          request for a container report
-   * @return container report
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  @Idempotent
-  public GetContainerReportResponse getContainerReport(
-      GetContainerReportRequest request) throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of Containers for an
-   * application attempt from the <code>ResourceManager</code>
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetContainersRequest} provides the
-   * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ResourceManager</code> verifies access to the
-   * method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ResourceManager</code> responds with a
-   * {@link GetContainersResponse} which includes a list of
-   * {@link ContainerReport} for all the containers of a specific application
-   * attempt.
-   * </p>
-   * 
-   * @param request
-   *          request for a list of container reports of an application attempt.
-   * @return reports on all containers of an application attempt
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  @Idempotent
-  public GetContainersResponse getContainers(GetContainersRequest request)
-      throws YarnException, IOException;
-
-  /**
-   * <p>
    * The interface used by clients to submit a new reservation to the
    * {@code ResourceManager}.
    * </p>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java
index 0bfd2ed..fc8e885 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationHistoryProtocol.java
@@ -18,37 +18,8 @@
 
 package org.apache.hadoop.yarn.api;
 
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.CancelDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
-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.api.records.Token;
-import org.apache.hadoop.yarn.exceptions.YarnException;
 
 /**
  * <p>
@@ -58,277 +29,5 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
  */
 @Public
 @Unstable
-public interface ApplicationHistoryProtocol {
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of an Application from the
-   * <code>ResourceManager</code>.
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetApplicationReportRequest} provides the
-   * {@link ApplicationId} of the application.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
-   * the application, queue etc. before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetApplicationReportResponse} which includes the
-   * {@link ApplicationReport} for the application.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access then the following
-   * fields in the report will be set to stubbed values:
-   * <ul>
-   * <li>host - set to "N/A"</li>
-   * <li>RPC port - set to -1</li>
-   * <li>client token - set to "N/A"</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL - set to "N/A"</li>
-   * <li>original tracking URL - set to "N/A"</li>
-   * <li>resource usage report - all values are -1</li>
-   * </ul>
-   * </p>
-   * 
-   * @param request
-   *          request for an application report
-   * @return application report
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetApplicationReportResponse getApplicationReport(
-      GetApplicationReportRequest request) throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of all Applications in the
-   * cluster from the <code>ApplicationHistoryServer</code>.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetApplicationsResponse} which includes a list of
-   * {@link ApplicationReport} for all the applications.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access for an application
-   * then the corresponding report will be filtered as described in
-   * {@link #getApplicationReport(GetApplicationReportRequest)}.
-   * </p>
-   * 
-   * @param request
-   *          request for reports on all the applications
-   * @return report on applications matching the given application types defined
-   *         in the request
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetApplicationsResponse
-      getApplications(GetApplicationsRequest request) throws YarnException,
-          IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of an Application Attempt
-   * from the <code>ApplicationHistoryServer</code>.
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetApplicationAttemptReportRequest} provides the
-   * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
-   * the method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetApplicationAttemptReportResponse} which includes the
-   * {@link ApplicationAttemptReport} for the application attempt.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access then the following
-   * fields in the report will be set to stubbed values:
-   * <ul>
-   * <li>host</li>
-   * <li>RPC port</li>
-   * <li>client token</li>
-   * <li>diagnostics - set to "N/A"</li>
-   * <li>tracking URL</li>
-   * </ul>
-   * </p>
-   * 
-   * @param request
-   *          request for an application attempt report
-   * @return application attempt report
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
-      GetApplicationAttemptReportRequest request) throws YarnException,
-      IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of all Application attempts
-   * in the cluster from the <code>ApplicationHistoryServer</code>.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetApplicationAttemptsRequest} which includes the
-   * {@link ApplicationAttemptReport} for all the applications attempts of a
-   * specified application attempt.
-   * </p>
-   * 
-   * <p>
-   * If the user does not have <code>VIEW_APP</code> access for an application
-   * then the corresponding report will be filtered as described in
-   * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
-   * </p>
-   * 
-   * @param request
-   *          request for reports on all application attempts of an application
-   * @return reports on all application attempts of an application
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetApplicationAttemptsResponse getApplicationAttempts(
-      GetApplicationAttemptsRequest request) throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of an Container from the
-   * <code>ApplicationHistoryServer</code>.
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetContainerReportRequest} provides the
-   * {@link ContainerId} of the container.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
-   * the method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetContainerReportResponse} which includes the
-   * {@link ContainerReport} for the container.
-   * </p>
-   * 
-   * @param request
-   *          request for a container report
-   * @return container report
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetContainerReportResponse getContainerReport(
-      GetContainerReportRequest request) throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get a report of Containers for an
-   * application attempt from the <code>ApplciationHistoryServer</code>.
-   * </p>
-   * 
-   * <p>
-   * The client, via {@link GetContainersRequest} provides the
-   * {@link ApplicationAttemptId} of the application attempt.
-   * </p>
-   * 
-   * <p>
-   * In secure mode,the <code>ApplicationHistoryServer</code> verifies access to
-   * the method before accepting the request.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with a
-   * {@link GetContainersResponse} which includes a list of
-   * {@link ContainerReport} for all the containers of a specific application
-   * attempt.
-   * </p>
-   * 
-   * @param request
-   *          request for a list of container reports of an application attempt.
-   * @return reports on all containers of an application attempt
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetContainersResponse getContainers(GetContainersRequest request)
-      throws YarnException, IOException;
-
-  /**
-   * <p>
-   * The interface used by clients to get delegation token, enabling the
-   * containers to be able to talk to the service using those tokens.
-   * </p>
-   * 
-   * <p>
-   * The <code>ApplicationHistoryServer</code> responds with the delegation
-   * token {@link Token} that can be used by the client to speak to this
-   * service.
-   * </p>
-   * 
-   * @param request
-   *          request to get a delegation token for the client.
-   * @return delegation token that can be used to talk to this service
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Public
-  @Unstable
-  public GetDelegationTokenResponse getDelegationToken(
-      GetDelegationTokenRequest request) throws YarnException, IOException;
-
-  /**
-   * Renew an existing delegation token.
-   * 
-   * @param request
-   *          the delegation token to be renewed.
-   * @return the new expiry time for the delegation token.
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Private
-  @Unstable
-  public RenewDelegationTokenResponse renewDelegationToken(
-      RenewDelegationTokenRequest request) throws YarnException, IOException;
-
-  /**
-   * Cancel an existing delegation token.
-   * 
-   * @param request
-   *          the delegation token to be cancelled.
-   * @return an empty response.
-   * @throws YarnException
-   * @throws IOException
-   */
-  @Private
-  @Unstable
-  public CancelDelegationTokenResponse cancelDelegationToken(
-      CancelDelegationTokenRequest request) throws YarnException, IOException;
+public interface ApplicationHistoryProtocol extends ApplicationBaseProtocol {
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
index 7e836b5..b04bc5d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/ResponseInfo.java
@@ -81,7 +81,11 @@ public class ResponseInfo implements Iterable<ResponseInfo.Item> {
   }
 
   public ResponseInfo _(String key, String url, Object anchor) {
-    items.add(Item.of(key, url, anchor));
+    if (url == null) {
+      items.add(Item.of(key, anchor, false));
+    } else {
+      items.add(Item.of(key, url, anchor));
+    }
     return this;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
index 91d2a20..055f895 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/YarnWebParams.java
@@ -22,6 +22,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public interface YarnWebParams {
+  static final String RM_WEB_UI = "ResourceManager";
+  static final String APP_HISTORY_WEB_UI = "ApplicationHistoryServer";
+  
   String NM_NODENAME = "nm.id";
   String APPLICATION_ID = "app.id";
   String APPLICATION_ATTEMPT_ID = "appattempt.id";
@@ -32,4 +35,5 @@ public interface YarnWebParams {
   String APP_STATE = "app.state";
   String QUEUE_NAME = "queue.name";
   String NODE_STATE = "node.state";
+  String WEB_UI_TYPE = "web.ui.type";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
index 6ee0d1c..a785c0c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/HtmlBlock.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.yarn.webapp.hamlet.Hamlet;
 @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"})
 public abstract class HtmlBlock extends TextView implements SubView {
 
+  protected static final String UNAVAILABLE = "N/A";
+
   public class Block extends Hamlet {
     Block(PrintWriter out, int level, boolean wasInline) {
       super(out, level, wasInline);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
index 2334fde..848224d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryClientService.java
@@ -49,30 +49,28 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetDelegationTokenResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RenewDelegationTokenResponse;
+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.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException;
-import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
-import org.apache.hadoop.yarn.exceptions.ContainerNotFoundException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.server.timeline.security.authorize.TimelinePolicyProvider;
 
-public class ApplicationHistoryClientService extends AbstractService {
+public class ApplicationHistoryClientService extends AbstractService implements
+    ApplicationHistoryProtocol {
   private static final Log LOG = LogFactory
     .getLog(ApplicationHistoryClientService.class);
   private ApplicationHistoryManager history;
-  private ApplicationHistoryProtocol protocolHandler;
   private Server server;
   private InetSocketAddress bindAddress;
 
   public ApplicationHistoryClientService(ApplicationHistoryManager history) {
     super("ApplicationHistoryClientService");
     this.history = history;
-    this.protocolHandler = new ApplicationHSClientProtocolHandler();
   }
 
   protected void serviceStart() throws Exception {
@@ -85,7 +83,7 @@ public class ApplicationHistoryClientService extends AbstractService {
         YarnConfiguration.DEFAULT_TIMELINE_SERVICE_PORT);
 
     server =
-        rpc.getServer(ApplicationHistoryProtocol.class, protocolHandler,
+        rpc.getServer(ApplicationHistoryProtocol.class, this,
           address, conf, null, conf.getInt(
             YarnConfiguration.TIMELINE_SERVICE_HANDLER_THREAD_COUNT,
             YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_THREAD_COUNT));
@@ -117,11 +115,6 @@ public class ApplicationHistoryClientService extends AbstractService {
   }
 
   @Private
-  public ApplicationHistoryProtocol getClientHandler() {
-    return this.protocolHandler;
-  }
-
-  @Private
   public InetSocketAddress getBindAddress() {
     return this.bindAddress;
   }
@@ -131,98 +124,97 @@ public class ApplicationHistoryClientService extends AbstractService {
     this.server.refreshServiceAcl(configuration, policyProvider);
   }
 
-  private class ApplicationHSClientProtocolHandler implements
-      ApplicationHistoryProtocol {
-
-    @Override
-    public CancelDelegationTokenResponse cancelDelegationToken(
-        CancelDelegationTokenRequest request) throws YarnException, IOException {
-      // TODO Auto-generated method stub
-      return null;
-    }
-
-    @Override
-    public GetApplicationAttemptReportResponse getApplicationAttemptReport(
-        GetApplicationAttemptReportRequest request) throws YarnException,
-        IOException {
-      try {
-        GetApplicationAttemptReportResponse response =
-            GetApplicationAttemptReportResponse.newInstance(history
-              .getApplicationAttempt(request.getApplicationAttemptId()));
-        return response;
-      } catch (IOException e) {
-        throw new ApplicationAttemptNotFoundException(e.getMessage());
-      }
-    }
+  @Override
+  public CancelDelegationTokenResponse cancelDelegationToken(
+      CancelDelegationTokenRequest request) throws YarnException, IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
 
-    @Override
-    public GetApplicationAttemptsResponse getApplicationAttempts(
-        GetApplicationAttemptsRequest request) throws YarnException,
-        IOException {
-      GetApplicationAttemptsResponse response =
-          GetApplicationAttemptsResponse
-            .newInstance(new ArrayList<ApplicationAttemptReport>(history
-              .getApplicationAttempts(request.getApplicationId()).values()));
+  @Override
+  public GetApplicationAttemptReportResponse getApplicationAttemptReport(
+      GetApplicationAttemptReportRequest request) throws YarnException,
+      IOException {
+    ApplicationAttemptId appAttemptId = request.getApplicationAttemptId();
+    try {
+      GetApplicationAttemptReportResponse response =
+          GetApplicationAttemptReportResponse.newInstance(history
+            .getApplicationAttempt(appAttemptId));
       return response;
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      throw e;
     }
+  }
 
-    @Override
-    public GetApplicationReportResponse getApplicationReport(
-        GetApplicationReportRequest request) throws YarnException, IOException {
-      try {
-        ApplicationId applicationId = request.getApplicationId();
-        GetApplicationReportResponse response =
-            GetApplicationReportResponse.newInstance(history
-              .getApplication(applicationId));
-        return response;
-      } catch (IOException e) {
-        throw new ApplicationNotFoundException(e.getMessage());
-      }
-    }
+  @Override
+  public GetApplicationAttemptsResponse getApplicationAttempts(
+      GetApplicationAttemptsRequest request) throws YarnException, IOException {
+    GetApplicationAttemptsResponse response =
+        GetApplicationAttemptsResponse
+          .newInstance(new ArrayList<ApplicationAttemptReport>(history
+            .getApplicationAttempts(request.getApplicationId()).values()));
+    return response;
+  }
 
-    @Override
-    public GetApplicationsResponse getApplications(
-        GetApplicationsRequest request) throws YarnException, IOException {
-      GetApplicationsResponse response =
-          GetApplicationsResponse.newInstance(new ArrayList<ApplicationReport>(
-            history.getAllApplications().values()));
+  @Override
+  public GetApplicationReportResponse getApplicationReport(
+      GetApplicationReportRequest request) throws YarnException, IOException {
+    ApplicationId applicationId = request.getApplicationId();
+    try {
+      GetApplicationReportResponse response =
+          GetApplicationReportResponse.newInstance(history
+            .getApplication(applicationId));
       return response;
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      throw e;
     }
+  }
 
-    @Override
-    public GetContainerReportResponse getContainerReport(
-        GetContainerReportRequest request) throws YarnException, IOException {
-      try {
-        GetContainerReportResponse response =
-            GetContainerReportResponse.newInstance(history.getContainer(request
-              .getContainerId()));
-        return response;
-      } catch (IOException e) {
-        throw new ContainerNotFoundException(e.getMessage());
-      }
-    }
+  @Override
+  public GetApplicationsResponse
+      getApplications(GetApplicationsRequest request) throws YarnException,
+          IOException {
+    GetApplicationsResponse response =
+        GetApplicationsResponse.newInstance(new ArrayList<ApplicationReport>(
+          history.getAllApplications().values()));
+    return response;
+  }
 
-    @Override
-    public GetContainersResponse getContainers(GetContainersRequest request)
-        throws YarnException, IOException {
-      GetContainersResponse response =
-          GetContainersResponse.newInstance(new ArrayList<ContainerReport>(
-            history.getContainers(request.getApplicationAttemptId()).values()));
+  @Override
+  public GetContainerReportResponse getContainerReport(
+      GetContainerReportRequest request) throws YarnException, IOException {
+    ContainerId containerId = request.getContainerId();
+    try {
+      GetContainerReportResponse response =
+          GetContainerReportResponse.newInstance(history
+            .getContainer(containerId));
       return response;
+    } catch (IOException e) {
+      LOG.error(e.getMessage(), e);
+      throw e;
     }
+  }
 
-    @Override
-    public GetDelegationTokenResponse getDelegationToken(
-        GetDelegationTokenRequest request) throws YarnException, IOException {
-      // TODO Auto-generated method stub
-      return null;
-    }
+  @Override
+  public GetContainersResponse getContainers(GetContainersRequest request)
+      throws YarnException, IOException {
+    GetContainersResponse response =
+        GetContainersResponse.newInstance(new ArrayList<ContainerReport>(
+          history.getContainers(request.getApplicationAttemptId()).values()));
+    return response;
+  }
 
-    @Override
-    public RenewDelegationTokenResponse renewDelegationToken(
-        RenewDelegationTokenRequest request) throws YarnException, IOException {
-      // TODO Auto-generated method stub
-      return null;
-    }
+  @Override
+  public GetDelegationTokenResponse getDelegationToken(
+      GetDelegationTokenRequest request) throws YarnException, IOException {
+    return null;
+  }
+
+  @Override
+  public RenewDelegationTokenResponse renewDelegationToken(
+      RenewDelegationTokenRequest request) throws YarnException, IOException {
+    return null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
index db25d29..041c31b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManager.java
@@ -18,11 +18,125 @@
 
 package org.apache.hadoop.yarn.server.applicationhistoryservice;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.yarn.server.api.ApplicationContext;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+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.exceptions.YarnException;
+
+@Private
+@Unstable
+public interface ApplicationHistoryManager {
+  /**
+   * This method returns Application {@link ApplicationReport} for the specified
+   * {@link ApplicationId}.
+   * 
+   * @param appId
+   * 
+   * @return {@link ApplicationReport} for the ApplicationId.
+   * @throws YarnException
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  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
+   */
+  @Public
+  @Unstable
+  Map<ContainerId, ContainerReport> getContainers(
+      ApplicationAttemptId appAttemptId) throws YarnException, IOException;
 
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public interface ApplicationHistoryManager extends ApplicationContext {
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
index c7e305c..55641ea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryServer.java
@@ -271,7 +271,7 @@ public class ApplicationHistoryServer extends CompositeService {
             .$for("applicationhistory", ApplicationHistoryClientService.class,
                 ahsClientService, "ws")
             .with(conf).at(bindAddress).start(
-                new AHSWebApp(timelineDataManager, historyManager));
+                new AHSWebApp(timelineDataManager, ahsClientService));
     } catch (Exception e) {
       String msg = "AHSWebApp failed to start.";
       LOG.error(msg, e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
index 4baa75d..152364e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSView.java
@@ -25,9 +25,8 @@ 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.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.view.TwoColumnLayout;
 
@@ -41,7 +40,7 @@ public class AHSView extends TwoColumnLayout {
   protected void preHead(Page.HTML<_> html) {
     commonPreHead(html);
     set(DATATABLES_ID, "apps");
-    set(initID(DATATABLES, "apps"), appsTableInit());
+    set(initID(DATATABLES, "apps"), WebPageUtils.appsTableInit());
     setTableStyles(html, "apps", ".queue {width:6em}", ".ui {width:8em}");
 
     // Set the correct title.
@@ -64,27 +63,4 @@ public class AHSView extends TwoColumnLayout {
   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();
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
index 814752b..4b579c6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebApp.java
@@ -19,8 +19,8 @@ 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.api.ApplicationBaseProtocol;
+import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryClientService;
 import org.apache.hadoop.yarn.server.timeline.TimelineDataManager;
 import org.apache.hadoop.yarn.server.timeline.webapp.TimelineWebServices;
 import org.apache.hadoop.yarn.webapp.GenericExceptionHandler;
@@ -30,17 +30,17 @@ import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
 public class AHSWebApp extends WebApp implements YarnWebParams {
 
-  private ApplicationHistoryManager applicationHistoryManager;
+  private final ApplicationHistoryClientService historyClientService;
   private TimelineDataManager timelineDataManager;
 
   public AHSWebApp(TimelineDataManager timelineDataManager,
-      ApplicationHistoryManager applicationHistoryManager) {
+      ApplicationHistoryClientService historyClientService) {
     this.timelineDataManager = timelineDataManager;
-    this.applicationHistoryManager = applicationHistoryManager;
+    this.historyClientService = historyClientService;
   }
 
-  public ApplicationHistoryManager getApplicationHistoryManager() {
-    return applicationHistoryManager;
+  public ApplicationHistoryClientService getApplicationHistoryClientService() {
+    return historyClientService;
   }
 
   public TimelineDataManager getTimelineDataManager() {
@@ -53,7 +53,7 @@ public class AHSWebApp extends WebApp implements YarnWebParams {
     bind(AHSWebServices.class);
     bind(TimelineWebServices.class);
     bind(GenericExceptionHandler.class);
-    bind(ApplicationContext.class).toInstance(applicationHistoryManager);
+    bind(ApplicationBaseProtocol.class).toInstance(historyClientService);
     bind(TimelineDataManager.class).toInstance(timelineDataManager);
     route("/", AHSController.class);
     route(pajoin("/apps", APP_STATE), AHSController.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
index 2040f57..2faba5f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AHSWebServices.java
@@ -32,7 +32,7 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 
 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.server.webapp.WebServices;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptInfo;
 import org.apache.hadoop.yarn.server.webapp.dao.AppAttemptsInfo;
@@ -50,8 +50,8 @@ import com.google.inject.Singleton;
 public class AHSWebServices extends WebServices {
 
   @Inject
-  public AHSWebServices(ApplicationContext appContext) {
-    super(appContext);
+  public AHSWebServices(ApplicationBaseProtocol appBaseProt) {
+    super(appBaseProt);
   }
 
   @GET

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
index 63b44bd..1e0a342 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppAttemptPage.java
@@ -21,9 +21,8 @@ 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.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
@@ -41,7 +40,7 @@ public class AppAttemptPage extends AHSView {
             $(YarnWebParams.APPLICATION_ATTEMPT_ID)));
 
     set(DATATABLES_ID, "containers");
-    set(initID(DATATABLES, "containers"), containersTableInit());
+    set(initID(DATATABLES, "containers"), WebPageUtils.containersTableInit());
     setTableStyles(html, "containers", ".queue {width:6em}", ".ui {width:8em}");
   }
 
@@ -50,16 +49,6 @@ public class AppAttemptPage extends AHSView {
     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]")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
index 96ca659..cf92c1d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/AppPage.java
@@ -22,9 +22,8 @@ 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.server.webapp.WebPageUtils;
 import org.apache.hadoop.yarn.webapp.SubView;
 import org.apache.hadoop.yarn.webapp.YarnWebParams;
 
@@ -40,9 +39,13 @@ public class AppPage extends AHSView {
       appId.isEmpty() ? "Bad request: missing application ID" : join(
         "Application ", $(YarnWebParams.APPLICATION_ID)));
 
-    set(DATATABLES_ID, "attempts");
-    set(initID(DATATABLES, "attempts"), attemptsTableInit());
+    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.APP_HISTORY_WEB_UI);
   }
 
   @Override
@@ -50,16 +53,6 @@ public class AppPage extends AHSView {
     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]")

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7ffdf7d1/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.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/TestApplicationHistoryClientService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
index 7c2593d..f0c4433 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
@@ -77,7 +77,7 @@ public class TestApplicationHistoryClientService {
     GetApplicationReportRequest request =
         GetApplicationReportRequest.newInstance(appId);
     GetApplicationReportResponse response =
-        clientService.getClientHandler().getApplicationReport(request);
+        clientService.getApplicationReport(request);
     ApplicationReport appReport = response.getApplicationReport();
     Assert.assertNotNull(appReport);
     Assert.assertEquals("application_0_0001", appReport.getApplicationId()
@@ -94,7 +94,7 @@ public class TestApplicationHistoryClientService {
     ApplicationId appId1 = ApplicationId.newInstance(0, 2);
     GetApplicationsRequest request = GetApplicationsRequest.newInstance();
     GetApplicationsResponse response =
-        clientService.getClientHandler().getApplications(request);
+        clientService.getApplications(request);
     List<ApplicationReport> appReport = response.getApplicationList();
     Assert.assertNotNull(appReport);
     Assert.assertEquals(appId, appReport.get(0).getApplicationId());
@@ -109,7 +109,7 @@ public class TestApplicationHistoryClientService {
     GetApplicationAttemptReportRequest request =
         GetApplicationAttemptReportRequest.newInstance(appAttemptId);
     GetApplicationAttemptReportResponse response =
-        clientService.getClientHandler().getApplicationAttemptReport(request);
+        clientService.getApplicationAttemptReport(request);
     ApplicationAttemptReport attemptReport =
         response.getApplicationAttemptReport();
     Assert.assertNotNull(attemptReport);
@@ -127,7 +127,7 @@ public class TestApplicationHistoryClientService {
     GetApplicationAttemptsRequest request =
         GetApplicationAttemptsRequest.newInstance(appId);
     GetApplicationAttemptsResponse response =
-        clientService.getClientHandler().getApplicationAttempts(request);
+        clientService.getApplicationAttempts(request);
     List<ApplicationAttemptReport> attemptReports =
         response.getApplicationAttemptList();
     Assert.assertNotNull(attemptReports);
@@ -146,7 +146,7 @@ public class TestApplicationHistoryClientService {
     GetContainerReportRequest request =
         GetContainerReportRequest.newInstance(containerId);
     GetContainerReportResponse response =
-        clientService.getClientHandler().getContainerReport(request);
+        clientService.getContainerReport(request);
     ContainerReport container = response.getContainerReport();
     Assert.assertNotNull(container);
     Assert.assertEquals(containerId, container.getContainerId());
@@ -165,7 +165,7 @@ public class TestApplicationHistoryClientService {
     GetContainersRequest request =
         GetContainersRequest.newInstance(appAttemptId);
     GetContainersResponse response =
-        clientService.getClientHandler().getContainers(request);
+        clientService.getContainers(request);
     List<ContainerReport> containers = response.getContainerList();
     Assert.assertNotNull(containers);
     Assert.assertEquals(containerId, containers.get(1).getContainerId());


[4/4] hadoop git commit: HDFS-7885. Datanode should not trust the generation stamp provided by client. Contributed by Tsz Wo Nicholas Sze.

Posted by vi...@apache.org.
HDFS-7885. Datanode should not trust the generation stamp provided by client. Contributed by Tsz Wo Nicholas Sze.

(cherry picked from commit 24db0812be64e83a48ade01fc1eaaeaedad4dec0)
(cherry picked from commit 994dadb9ba0a3b87b6548e6e0801eadd26554d55)
(cherry picked from commit 0bc5c6495a7feb4365af0ce5fe48fc87b7e1749f)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b25491dc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b25491dc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b25491dc

Branch: refs/heads/branch-2.6.1
Commit: b25491dc45be92cc3ec157d998a2632387c3952f
Parents: 7ffdf7d
Author: Jing Zhao <ji...@apache.org>
Authored: Fri Mar 6 10:55:56 2015 -0800
Committer: Vinod Kumar Vavilapalli <vi...@apache.org>
Committed: Tue Sep 1 15:17:57 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../datanode/fsdataset/impl/FsDatasetImpl.java  | 15 +++++
 .../hadoop/hdfs/TestBlockReaderLocalLegacy.java | 63 ++++++++++++++++++++
 3 files changed, 81 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b25491dc/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1811337..d7ff237 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -85,6 +85,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7871. NameNodeEditLogRoller can keep printing "Swallowing exception"
     message. (jing9)
 
+    HDFS-7885. Datanode should not trust the generation stamp provided by
+    client. (Tsz Wo Nicholas Sze via jing9)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b25491dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 0d9f096..0c2337e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -2276,6 +2276,21 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
   @Override // FsDatasetSpi
   public BlockLocalPathInfo getBlockLocalPathInfo(ExtendedBlock block)
       throws IOException {
+    synchronized(this) {
+      final Replica replica = volumeMap.get(block.getBlockPoolId(),
+          block.getBlockId());
+      if (replica == null) {
+        throw new ReplicaNotFoundException(block);
+      }
+      if (replica.getGenerationStamp() < block.getGenerationStamp()) {
+        throw new IOException(
+            "Replica generation stamp < block generation stamp, block="
+            + block + ", replica=" + replica);
+      } else if (replica.getGenerationStamp() > block.getGenerationStamp()) {
+        block.setGenerationStamp(replica.getGenerationStamp());
+      }
+    }
+
     File datafile = getBlockFile(block);
     File metafile = FsDatasetUtil.getMetaFile(datafile, block.getGenerationStamp());
     BlockLocalPathInfo info = new BlockLocalPathInfo(block,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b25491dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
index cb50539..1c4134f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderLocalLegacy.java
@@ -30,11 +30,16 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.BlockLocalPathInfo;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.unix.DomainSocket;
 import org.apache.hadoop.net.unix.TemporarySocketDirectory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.BeforeClass;
@@ -153,4 +158,62 @@ public class TestBlockReaderLocalLegacy {
     Arrays.equals(orig, buf);
     cluster.shutdown();
   }
+
+  @Test(timeout=20000)
+  public void testBlockReaderLocalLegacyWithAppend() throws Exception {
+    final short REPL_FACTOR = 1;
+    final HdfsConfiguration conf = getConfiguration(null);
+    conf.setBoolean(DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL, true);
+
+    final MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    cluster.waitActive();
+
+    final DistributedFileSystem dfs = cluster.getFileSystem();
+    final Path path = new Path("/testBlockReaderLocalLegacy");
+    DFSTestUtil.createFile(dfs, path, 10, REPL_FACTOR, 0);
+    DFSTestUtil.waitReplication(dfs, path, REPL_FACTOR);
+
+    final ClientDatanodeProtocol proxy;
+    final Token<BlockTokenIdentifier> token;
+    final ExtendedBlock originalBlock;
+    final long originalGS;
+    {
+      final LocatedBlock lb = cluster.getNameNode().getRpcServer()
+          .getBlockLocations(path.toString(), 0, 1).get(0);
+      proxy = DFSUtil.createClientDatanodeProtocolProxy(
+          lb.getLocations()[0], conf, 60000, false);
+      token = lb.getBlockToken();
+
+      // get block and generation stamp
+      final ExtendedBlock blk = new ExtendedBlock(lb.getBlock());
+      originalBlock = new ExtendedBlock(blk);
+      originalGS = originalBlock.getGenerationStamp();
+
+      // test getBlockLocalPathInfo
+      final BlockLocalPathInfo info = proxy.getBlockLocalPathInfo(blk, token);
+      Assert.assertEquals(originalGS, info.getBlock().getGenerationStamp());
+    }
+
+    { // append one byte
+      FSDataOutputStream out = dfs.append(path);
+      out.write(1);
+      out.close();
+    }
+
+    {
+      // get new generation stamp
+      final LocatedBlock lb = cluster.getNameNode().getRpcServer()
+          .getBlockLocations(path.toString(), 0, 1).get(0);
+      final long newGS = lb.getBlock().getGenerationStamp();
+      Assert.assertTrue(newGS > originalGS);
+
+      // getBlockLocalPathInfo using the original block.
+      Assert.assertEquals(originalGS, originalBlock.getGenerationStamp());
+      final BlockLocalPathInfo info = proxy.getBlockLocalPathInfo(
+          originalBlock, token);
+      Assert.assertEquals(newGS, info.getBlock().getGenerationStamp());
+    }
+    cluster.shutdown();
+  }
 }