From 7ffdf7d105fd2ce7a484cb96a96f414670bec141 Mon Sep 17 00:00:00 2001
From: Jian He
Date: Thu, 5 Mar 2015 21:14:41 -0800
Subject: [PATCH] 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)
---
hadoop-yarn-project/CHANGES.txt | 3 +
.../yarn/api/ApplicationBaseProtocol.java | 355 ++++++++++++++++++
.../yarn/api/ApplicationClientProtocol.java | 290 +-------------
.../yarn/api/ApplicationHistoryProtocol.java | 303 +--------------
.../hadoop/yarn/webapp/ResponseInfo.java | 6 +-
.../hadoop/yarn/webapp/YarnWebParams.java | 4 +
.../hadoop/yarn/webapp/view/HtmlBlock.java | 2 +
.../ApplicationHistoryClientService.java | 196 +++++-----
.../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 +-
.../yarn/server/webapp/WebPageUtils.java | 86 +++++
.../yarn/server/webapp/WebServices.java | 68 +++-
.../yarn/server/webapp/dao/AppInfo.java | 11 +-
.../webapp/AppAttemptPage.java | 55 +++
.../resourcemanager/webapp/AppBlock.java | 265 -------------
.../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 +-
.../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, 1311 insertions(+), 1474 deletions(-)
create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationBaseProtocol.java
delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationContext.java
create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/webapp/WebPageUtils.java
create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppAttemptPage.java
delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppBlock.java
delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/AppsBlock.java
create mode 100644 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/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5317b594304..b8c15f93263 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
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 00000000000..2a8a2833826
--- /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;
+
+/**
+ *
+ * The protocol between clients and the ResourceManager
or
+ * ApplicationHistoryServer
to get information on applications,
+ * application attempts and containers.
+ *
+ *
+ */
+@Private
+@Unstable
+public interface ApplicationBaseProtocol {
+
+ /**
+ *
+ * The interface used by clients to get a report of an Application from the
+ * ResourceManager
or ApplicationHistoryServer
.
+ *
+ *
+ *
+ * The client, via {@link GetApplicationReportRequest} provides the
+ * {@link ApplicationId} of the application.
+ *
+ *
+ *
+ * In secure mode,the ResourceManager
or
+ * ApplicationHistoryServer
verifies access to the application,
+ * queue etc. before accepting the request.
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetApplicationReportResponse} which includes the
+ * {@link ApplicationReport} for the application.
+ *
+ *
+ *
+ * If the user does not have VIEW_APP
access then the following
+ * fields in the report will be set to stubbed values:
+ *
+ * - host - set to "N/A"
+ * - RPC port - set to -1
+ * - client token - set to "N/A"
+ * - diagnostics - set to "N/A"
+ * - tracking URL - set to "N/A"
+ * - original tracking URL - set to "N/A"
+ * - resource usage report - all values are -1
+ *
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get a report of Applications matching the
+ * filters defined by {@link GetApplicationsRequest} in the cluster from the
+ * ResourceManager
or ApplicationHistoryServer
.
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetApplicationsResponse} which includes the
+ * {@link ApplicationReport} for the applications.
+ *
+ *
+ *
+ * If the user does not have VIEW_APP
access for an application
+ * then the corresponding report will be filtered as described in
+ * {@link #getApplicationReport(GetApplicationReportRequest)}.
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get a report of an Application Attempt
+ * from the ResourceManager
or
+ * ApplicationHistoryServer
+ *
+ *
+ *
+ * The client, via {@link GetApplicationAttemptReportRequest} provides the
+ * {@link ApplicationAttemptId} of the application attempt.
+ *
+ *
+ *
+ * In secure mode,the ResourceManager
or
+ * ApplicationHistoryServer
verifies access to the method before
+ * accepting the request.
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetApplicationAttemptReportResponse} which includes
+ * the {@link ApplicationAttemptReport} for the application attempt.
+ *
+ *
+ *
+ * If the user does not have VIEW_APP
access then the following
+ * fields in the report will be set to stubbed values:
+ *
+ * - host
+ * - RPC port
+ * - client token
+ * - diagnostics - set to "N/A"
+ * - tracking URL
+ *
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get a report of all Application attempts
+ * in the cluster from the ResourceManager
or
+ * ApplicationHistoryServer
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetApplicationAttemptsRequest} which includes the
+ * {@link ApplicationAttemptReport} for all the applications attempts of a
+ * specified application attempt.
+ *
+ *
+ *
+ * If the user does not have VIEW_APP
access for an application
+ * then the corresponding report will be filtered as described in
+ * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get a report of an Container from the
+ * ResourceManager
or ApplicationHistoryServer
+ *
+ *
+ *
+ * The client, via {@link GetContainerReportRequest} provides the
+ * {@link ContainerId} of the container.
+ *
+ *
+ *
+ * In secure mode,the ResourceManager
or
+ * ApplicationHistoryServer
verifies access to the method before
+ * accepting the request.
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetContainerReportResponse} which includes the
+ * {@link ContainerReport} for the container.
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get a report of Containers for an
+ * application attempt from the ResourceManager
or
+ * ApplicationHistoryServer
+ *
+ *
+ *
+ * The client, via {@link GetContainersRequest} provides the
+ * {@link ApplicationAttemptId} of the application attempt.
+ *
+ *
+ *
+ * In secure mode,the ResourceManager
or
+ * ApplicationHistoryServer
verifies access to the method before
+ * accepting the request.
+ *
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * responds with a {@link GetContainersResponse} which includes a list of
+ * {@link ContainerReport} for all the containers of a specific application
+ * attempt.
+ *
+ *
+ * @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;
+
+ /**
+ *
+ * The interface used by clients to get delegation token, enabling the
+ * containers to be able to talk to the service using those tokens.
+ *
+ *
+ * The ResourceManager
or ApplicationHistoryServer
+ * 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;
+
+}
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 add35f1ac6c..004b9aab231 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 {
/**
*
The interface used by clients to obtain a new {@link ApplicationId} for
* submitting new applications.
@@ -196,44 +172,6 @@ public interface ApplicationClientProtocol {
KillApplicationRequest request)
throws YarnException, IOException;
- /**
- * The interface used by clients to get a report of an Application from
- * the ResourceManager
.
- *
- * The client, via {@link GetApplicationReportRequest} provides the
- * {@link ApplicationId} of the application.
- *
- * In secure mode,the ResourceManager
verifies access to the
- * application, queue etc. before accepting the request.
- *
- * The ResourceManager
responds with a
- * {@link GetApplicationReportResponse} which includes the
- * {@link ApplicationReport} for the application.
- *
- * If the user does not have VIEW_APP
access then the
- * following fields in the report will be set to stubbed values:
- *
- * - host - set to "N/A"
- * - RPC port - set to -1
- * - client token - set to "N/A"
- * - diagnostics - set to "N/A"
- * - tracking URL - set to "N/A"
- * - original tracking URL - set to "N/A"
- * - resource usage report - all values are -1
- *
- *
- * @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;
-
/**
* The interface used by clients to get metrics about the cluster from
* the ResourceManager
.
@@ -254,35 +192,7 @@ public interface ApplicationClientProtocol {
public GetClusterMetricsResponse getClusterMetrics(
GetClusterMetricsRequest request)
throws YarnException, IOException;
-
- /**
- * The interface used by clients to get a report of Applications
- * matching the filters defined by {@link GetApplicationsRequest}
- * in the cluster from the ResourceManager
.
- *
- * The ResourceManager
responds with a
- * {@link GetApplicationsResponse} which includes the
- * {@link ApplicationReport} for the applications.
- *
- * If the user does not have VIEW_APP
access for an
- * application then the corresponding report will be filtered as
- * described in {@link #getApplicationReport(GetApplicationReportRequest)}.
- *
- *
- * @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;
-
+
/**
* The interface used by clients to get a report of all nodes
* in the cluster from the ResourceManager
.
@@ -344,56 +254,7 @@ public interface ApplicationClientProtocol {
public GetQueueUserAclsInfoResponse getQueueUserAcls(
GetQueueUserAclsInfoRequest request)
throws YarnException, IOException;
-
- /**
- * The interface used by clients to get delegation token, enabling the
- * containers to be able to talk to the service using those tokens.
- *
- *
The ResourceManager
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.
*
@@ -408,153 +269,6 @@ public interface ApplicationClientProtocol {
public MoveApplicationAcrossQueuesResponse moveApplicationAcrossQueues(
MoveApplicationAcrossQueuesRequest request) throws YarnException, IOException;
- /**
- *
- * The interface used by clients to get a report of an Application Attempt
- * from the ResourceManager
- *
- *
- *
- * The client, via {@link GetApplicationAttemptReportRequest} provides the
- * {@link ApplicationAttemptId} of the application attempt.
- *
- *
- *
- * In secure mode,the ResourceManager
verifies access to
- * the method before accepting the request.
- *
- *
- *
- * The ResourceManager
responds with a
- * {@link GetApplicationAttemptReportResponse} which includes the
- * {@link ApplicationAttemptReport} for the application attempt.
- *
- *
- *
- * If the user does not have VIEW_APP
access then the following
- * fields in the report will be set to stubbed values:
- *
- * - host
- * - RPC port
- * - client token
- * - diagnostics - set to "N/A"
- * - tracking URL
- *
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of all Application attempts
- * in the cluster from the ResourceManager
- *
- *
- *
- * The ResourceManager
responds with a
- * {@link GetApplicationAttemptsRequest} which includes the
- * {@link ApplicationAttemptReport} for all the applications attempts of a
- * specified application attempt.
- *
- *
- *
- * If the user does not have VIEW_APP
access for an application
- * then the corresponding report will be filtered as described in
- * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of an Container from the
- * ResourceManager
- *
- *
- *
- * The client, via {@link GetContainerReportRequest} provides the
- * {@link ContainerId} of the container.
- *
- *
- *
- * In secure mode,the ResourceManager
verifies access to the
- * method before accepting the request.
- *
- *
- *
- * The ResourceManager
responds with a
- * {@link GetContainerReportResponse} which includes the
- * {@link ContainerReport} for the container.
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of Containers for an
- * application attempt from the ResourceManager
- *
- *
- *
- * The client, via {@link GetContainersRequest} provides the
- * {@link ApplicationAttemptId} of the application attempt.
- *
- *
- *
- * In secure mode,the ResourceManager
verifies access to the
- * method before accepting the request.
- *
- *
- *
- * The ResourceManager
responds with a
- * {@link GetContainersResponse} which includes a list of
- * {@link ContainerReport} for all the containers of a specific application
- * attempt.
- *
- *
- * @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;
-
/**
*
* The interface used by clients to submit a new reservation to the
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 0bfd2eda47e..fc8e8850f68 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;
/**
*
@@ -58,277 +29,5 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
*/
@Public
@Unstable
-public interface ApplicationHistoryProtocol {
-
- /**
- *
- * The interface used by clients to get a report of an Application from the
- * ResourceManager
.
- *
- *
- *
- * The client, via {@link GetApplicationReportRequest} provides the
- * {@link ApplicationId} of the application.
- *
- *
- *
- * In secure mode,the ApplicationHistoryServer
verifies access to
- * the application, queue etc. before accepting the request.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetApplicationReportResponse} which includes the
- * {@link ApplicationReport} for the application.
- *
- *
- *
- * If the user does not have VIEW_APP
access then the following
- * fields in the report will be set to stubbed values:
- *
- * - host - set to "N/A"
- * - RPC port - set to -1
- * - client token - set to "N/A"
- * - diagnostics - set to "N/A"
- * - tracking URL - set to "N/A"
- * - original tracking URL - set to "N/A"
- * - resource usage report - all values are -1
- *
- *
- *
- * @param request
- * request for an application report
- * @return application report
- * @throws YarnException
- * @throws IOException
- */
- @Public
- @Unstable
- public GetApplicationReportResponse getApplicationReport(
- GetApplicationReportRequest request) throws YarnException, IOException;
-
- /**
- *
- * The interface used by clients to get a report of all Applications in the
- * cluster from the ApplicationHistoryServer
.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetApplicationsResponse} which includes a list of
- * {@link ApplicationReport} for all the applications.
- *
- *
- *
- * If the user does not have VIEW_APP
access for an application
- * then the corresponding report will be filtered as described in
- * {@link #getApplicationReport(GetApplicationReportRequest)}.
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of an Application Attempt
- * from the ApplicationHistoryServer
.
- *
- *
- *
- * The client, via {@link GetApplicationAttemptReportRequest} provides the
- * {@link ApplicationAttemptId} of the application attempt.
- *
- *
- *
- * In secure mode,the ApplicationHistoryServer
verifies access to
- * the method before accepting the request.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetApplicationAttemptReportResponse} which includes the
- * {@link ApplicationAttemptReport} for the application attempt.
- *
- *
- *
- * If the user does not have VIEW_APP
access then the following
- * fields in the report will be set to stubbed values:
- *
- * - host
- * - RPC port
- * - client token
- * - diagnostics - set to "N/A"
- * - tracking URL
- *
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of all Application attempts
- * in the cluster from the ApplicationHistoryServer
.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetApplicationAttemptsRequest} which includes the
- * {@link ApplicationAttemptReport} for all the applications attempts of a
- * specified application attempt.
- *
- *
- *
- * If the user does not have VIEW_APP
access for an application
- * then the corresponding report will be filtered as described in
- * {@link #getApplicationAttemptReport(GetApplicationAttemptReportRequest)}.
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get a report of an Container from the
- * ApplicationHistoryServer
.
- *
- *
- *
- * The client, via {@link GetContainerReportRequest} provides the
- * {@link ContainerId} of the container.
- *
- *
- *
- * In secure mode,the ApplicationHistoryServer
verifies access to
- * the method before accepting the request.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetContainerReportResponse} which includes the
- * {@link ContainerReport} for the container.
- *
- *
- * @param request
- * request for a container report
- * @return container report
- * @throws YarnException
- * @throws IOException
- */
- @Public
- @Unstable
- public GetContainerReportResponse getContainerReport(
- GetContainerReportRequest request) throws YarnException, IOException;
-
- /**
- *
- * The interface used by clients to get a report of Containers for an
- * application attempt from the ApplciationHistoryServer
.
- *
- *
- *
- * The client, via {@link GetContainersRequest} provides the
- * {@link ApplicationAttemptId} of the application attempt.
- *
- *
- *
- * In secure mode,the ApplicationHistoryServer
verifies access to
- * the method before accepting the request.
- *
- *
- *
- * The ApplicationHistoryServer
responds with a
- * {@link GetContainersResponse} which includes a list of
- * {@link ContainerReport} for all the containers of a specific application
- * attempt.
- *
- *
- * @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;
-
- /**
- *
- * The interface used by clients to get delegation token, enabling the
- * containers to be able to talk to the service using those tokens.
- *
- *
- *
- * The ApplicationHistoryServer
responds with the delegation
- * token {@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
- @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 {
}
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 7e836b5bac4..b04bc5dd1be 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 {
}
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;
}
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 91d2a2019ab..055f895a904 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";
}
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 6ee0d1c9c32..a785c0c73d0 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);
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 2334fdea16a..848224d39df 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));
@@ -116,11 +114,6 @@ public class ApplicationHistoryClientService extends AbstractService {
super.serviceStop();
}
- @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 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 GetApplicationAttemptsResponse getApplicationAttempts(
- GetApplicationAttemptsRequest request) throws YarnException,
- IOException {
- GetApplicationAttemptsResponse response =
- GetApplicationAttemptsResponse
- .newInstance(new ArrayList(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;
- }
-
- @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 GetApplicationsResponse getApplications(
- GetApplicationsRequest request) throws YarnException, IOException {
- GetApplicationsResponse response =
- GetApplicationsResponse.newInstance(new ArrayList(
- history.getAllApplications().values()));
- return response;
- }
-
- @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 GetContainersResponse getContainers(GetContainersRequest request)
- throws YarnException, IOException {
- GetContainersResponse response =
- GetContainersResponse.newInstance(new ArrayList(
- history.getContainers(request.getApplicationAttemptId()).values()));
- return response;
- }
-
- @Override
- public GetDelegationTokenResponse getDelegationToken(
- GetDelegationTokenRequest request) throws YarnException, IOException {
- // TODO Auto-generated method stub
- return null;
- }
-
- @Override
- public RenewDelegationTokenResponse renewDelegationToken(
- RenewDelegationTokenRequest request) throws YarnException, IOException {
- // TODO Auto-generated method stub
- return null;
+ } catch (IOException e) {
+ LOG.error(e.getMessage(), e);
+ throw e;
}
}
+
+ @Override
+ public GetApplicationAttemptsResponse getApplicationAttempts(
+ GetApplicationAttemptsRequest request) throws YarnException, IOException {
+ GetApplicationAttemptsResponse response =
+ GetApplicationAttemptsResponse
+ .newInstance(new ArrayList(history
+ .getApplicationAttempts(request.getApplicationId()).values()));
+ return response;
+ }
+
+ @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 GetApplicationsResponse
+ getApplications(GetApplicationsRequest request) throws YarnException,
+ IOException {
+ GetApplicationsResponse response =
+ GetApplicationsResponse.newInstance(new ArrayList(
+ history.getAllApplications().values()));
+ return response;
+ }
+
+ @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 GetContainersResponse getContainers(GetContainersRequest request)
+ throws YarnException, IOException {
+ GetContainersResponse response =
+ GetContainersResponse.newInstance(new ArrayList(
+ history.getContainers(request.getApplicationAttemptId()).values()));
+ return response;
+ }
+
+ @Override
+ public GetDelegationTokenResponse getDelegationToken(
+ GetDelegationTokenRequest request) throws YarnException, IOException {
+ return null;
+ }
+
+ @Override
+ public RenewDelegationTokenResponse renewDelegationToken(
+ RenewDelegationTokenRequest request) throws YarnException, IOException {
+ return null;
+ }
}
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 db25d298b3c..041c31bc376 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 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 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 getContainers(
+ ApplicationAttemptId appAttemptId) throws YarnException, IOException;
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public interface ApplicationHistoryManager extends ApplicationContext {
}
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 c7e305cb214..55641ead06b 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);
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 4baa75d1bdd..152364e8722 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();
- }
}
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 814752bcf54..4b579c60e56 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);
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 2040f575141..2faba5f30da 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
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 63b44bde663..1e0a3421ae7 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]")
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 96ca65918b0..cf92c1db9af 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]")
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 7c2593d9e0a..f0c44335459 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 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 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 containers = response.getContainerList();
Assert.assertNotNull(containers);
Assert.assertEquals(containerId, containers.get(1).getContainerId());
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 7bac6f265c2..2cd75800068 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 {
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 76bf8c3c755..ebb2ec84bc6 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"));
}
-
}
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 0e2ffdfd909..00000000000
--- 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 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 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 getContainers(
- ApplicationAttemptId appAttemptId) throws YarnException, IOException;
-}
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 4a02892f7e2..ea33f4f3f36 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 () {
@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 containers = null;
+ try {
+ final GetContainersRequest request =
+ GetContainersRequest.newInstance(appAttemptId);
+ if (callerUGI == null) {
+ containers = appBaseProt.getContainers(request).getContainerList();
+ } else {
+ containers = callerUGI.doAs(
+ new PrivilegedExceptionAction> () {
+ @Override
+ public Collection 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 containers;
- try {
- if (callerUGI == null) {
- containers = appContext.getContainers(appAttemptId).values();
- } else {
- containers = callerUGI.doAs(
- new PrivilegedExceptionAction> () {
- @Override
- public Collection 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("\",\"")
- .append(
- StringEscapeUtils.escapeJavaScript(StringEscapeUtils
- .escapeHtml(container.getAssignedNodeId()))).append("\",\"")
+ .append(container.getAssignedNodeId() == null ? "N/A" :
+ StringEscapeUtils.escapeJavaScript(StringEscapeUtils
+ .escapeHtml(container.getAssignedNodeId())))
+ .append("\",\"")
.append(container.getContainerExitStatus()).append("\",\"")
@@ -187,4 +214,14 @@ public class AppAttemptBlock extends HtmlBlock {
tbody._()._();
}
+
+ private boolean hasAMContainer(ContainerId containerId,
+ Collection containers) {
+ for (ContainerReport container : containers) {
+ if (containerId.equals(container.getContainerId())) {
+ return true;
+ }
+ }
+ return false;
+ }
}
\ No newline at end of file
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 8fa40860b82..37b60bbc4ce 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 () {
@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 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> () {
@Override
public Collection 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 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> 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 () {
@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("\",\"")
.append(startTime)
.append("\",\"")
- .append(
- nodeLink == null ? "N/A" : StringEscapeUtils
+ .append(nodeLink == null ? "N/A" : StringEscapeUtils
.escapeJavaScript(StringEscapeUtils.escapeHtml(nodeLink)))
- .append("\",\"")
+ .append("\",\"")
.append(logsLink == null ? "N/A" : "Logs").append("\"],\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();
+ }
}
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 f341cf6e55d..161486df1b2 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 reqAppStates = null;
+ EnumSet reqAppStates =
+ EnumSet.noneOf(YarnApplicationState.class);
String reqStateString = $(APP_STATE);
if (reqStateString != null && !reqStateString.isEmpty()) {
String[] appStateStrings = reqStateString.split(",");
- reqAppStates = new HashSet(appStateStrings.length);
for (String stateString : appStateStrings) {
- reqAppStates.add(YarnApplicationState.valueOf(stateString));
+ reqAppStates.add(YarnApplicationState.valueOf(stateString.trim()));
}
}
UserGroupInformation callerUGI = getCallerUGI();
- Collection appReports;
+ Collection 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> () {
@Override
public Collection 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("[\" ").append("
").append("\",\" ").append("\",\"").append("History")
- .append("\"],\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("\"],\n");
}
if (appsTableData.charAt(appsTableData.length() - 2) == ',') {
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 2bb48a8b1cc..ed50c7af671 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 () {
@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);
}
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 00000000000..384a976e457
--- /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
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 385d10ae8b0..a02b80fb670 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 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> () {
@Override
public Collection 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 () {
@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 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> () {
@Override
public Collection 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 () {
@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 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> () {
@Override
public Collection 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 () {
@Override
public ContainerReport run() throws Exception {
- return appContext.getContainer(cid);
+ GetContainerReportRequest request =
+ GetContainerReportRequest.newInstance(cid);
+ return appBaseProt.getContainerReport(request).getContainerReport();
}
});
}
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 d78f9287ffb..e8b1acce498 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;
+ }
}
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 00000000000..92eae48485d
--- /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
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 bae4947ec52..00000000000
--- 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 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 attempts = rmApp.getAppAttempts().values();
- String amString =
- attempts.size() == 1 ? "ApplicationMaster" : "ApplicationMasters";
-
- DIV div = html.
- _(InfoBlock.class).
- div(_INFO_WRAP);
- // MRAppMasters Table
- TABLE> 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();
- }
-}
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 a55c62f4b59..77ae44c8d38 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;
}
}
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 054a1a7020e..00000000000
--- 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
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> 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 reqAppStates = null;
- String reqStateString = $(APP_STATE);
- if (reqStateString != null && !reqStateString.isEmpty()) {
- String[] appStateStrings = reqStateString.split(",");
- reqAppStates = new HashSet(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("[\"")
- .append(appInfo.getAppId()).append("\",\"")
- .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("
")
- .append("\",\"")
- .append(appInfo.getTrackingUI()).append("\"],\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._()._();
- }
-}
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 6d461f659c7..cb0836a7e5f 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;
/**
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 89b4a78430c..ffead488b11 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;
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 00000000000..b8cd1adbd58
--- /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
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 e05987bb594..1c8828cf7e0 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;
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 0d3c29763bc..2902cc7e9f7 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();
- }
}
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 67c73b81273..066ba520725 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
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 f186bf498b0..e65c57d93c3 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);
}
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 769c4da6921..1a437f834b7 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();
}
}
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 9732c193b51..8c7b14d859a 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);
}
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 abc701db192..f640cc8df92 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 appReports = new ArrayList();
+ 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"});
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 f07cb8d2f15..f03f2c43229 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);
+ }
}