in hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppInfo.java [147:364]
public AppInfo(ResourceManager rm, RMApp app, Boolean hasAccess,
String schemePrefix, DeSelectFields deSelects) {
this.schemePrefix = schemePrefix;
if (app != null) {
String trackingUrl = app.getTrackingUrl();
this.state = app.createApplicationState();
this.trackingUrlIsNotReady = trackingUrl == null || trackingUrl.isEmpty()
|| YarnApplicationState.NEW == this.state
|| YarnApplicationState.NEW_SAVING == this.state
|| YarnApplicationState.SUBMITTED == this.state
|| YarnApplicationState.ACCEPTED == this.state;
this.trackingUI = this.trackingUrlIsNotReady ? "UNASSIGNED"
: (app.getFinishTime() == 0 ? "ApplicationMaster" : "History");
if (!trackingUrlIsNotReady) {
this.trackingUrl =
WebAppUtils.getURLWithScheme(schemePrefix, trackingUrl);
this.trackingUrlPretty = this.trackingUrl;
} else {
this.trackingUrlPretty = "UNASSIGNED";
}
this.applicationId = app.getApplicationId();
this.applicationType = app.getApplicationType();
this.appIdNum = String.valueOf(app.getApplicationId().getId());
this.id = app.getApplicationId().toString();
this.user = app.getUser().toString();
this.name = app.getName().toString();
this.queue = app.getQueue().toString();
this.priority = 0;
this.masterNodeId = "";
if (app.getApplicationPriority() != null) {
this.priority = app.getApplicationPriority().getPriority();
}
this.progress = app.getProgress() * 100;
this.diagnostics = app.getDiagnostics().toString();
if (diagnostics == null || diagnostics.isEmpty()) {
this.diagnostics = "";
}
if (app.getApplicationTags() != null
&& !app.getApplicationTags().isEmpty()) {
this.applicationTags = Joiner.on(',').join(app.getApplicationTags());
}
this.finalStatus = app.getFinalApplicationStatus();
this.clusterId = ResourceManager.getClusterTimeStamp();
if (hasAccess) {
if (rm != null && rm.getConfig() != null) {
try {
Configuration yarnConfig = rm.getConfig();
subClusterId = new SubClusterIdInfo(YarnConfiguration.getClusterId(yarnConfig));
rmClusterId = this.subClusterId.toId().toString();
} catch (HadoopIllegalArgumentException e) {
subClusterId = null;
rmClusterId = null;
}
}
this.startedTime = app.getStartTime();
this.launchTime = app.getLaunchTime();
this.finishedTime = app.getFinishTime();
this.elapsedTime =
Times.elapsed(app.getStartTime(), app.getFinishTime());
this.logAggregationStatus = app.getLogAggregationStatusForAppReport();
RMAppAttempt attempt = app.getCurrentAppAttempt();
if (attempt != null) {
Container masterContainer = attempt.getMasterContainer();
if (masterContainer != null) {
this.amContainerLogsExist = true;
this.amContainerLogs = WebAppUtils.getRunningLogURL(
schemePrefix + masterContainer.getNodeHttpAddress(),
masterContainer.getId().toString(), app.getUser());
this.amHostHttpAddress = masterContainer.getNodeHttpAddress();
this.masterNodeId = masterContainer.getNodeId().toString();
}
this.amRPCAddress = getAmRPCAddressFromRMAppAttempt(attempt);
ApplicationResourceUsageReport resourceReport =
attempt.getApplicationResourceUsageReport();
if (resourceReport != null) {
Resource usedResources = resourceReport.getUsedResources();
Resource reservedResources = resourceReport.getReservedResources();
allocatedMB = usedResources.getMemorySize();
allocatedVCores = usedResources.getVirtualCores();
reservedMB = reservedResources.getMemorySize();
reservedVCores = reservedResources.getVirtualCores();
runningContainers = resourceReport.getNumUsedContainers();
queueUsagePercentage = resourceReport.getQueueUsagePercentage();
clusterUsagePercentage = resourceReport.getClusterUsagePercentage();
}
/*
* When the deSelects parameter contains "resourceRequests", it skips
* returning massive ResourceRequest objects and vice versa. Default
* behavior is no skipping. (YARN-6280)
*/
if (!deSelects.contains(DeSelectType.RESOURCE_REQUESTS)) {
List<ResourceRequest> resourceRequestsRaw = rm.getRMContext()
.getScheduler().getPendingResourceRequestsForAttempt(
attempt.getAppAttemptId());
if (resourceRequestsRaw != null) {
for (ResourceRequest req : resourceRequestsRaw) {
resourceRequests.add(new ResourceRequestInfo(req));
}
}
List<SchedulingRequest> schedulingRequestsRaw = rm.getRMContext()
.getScheduler().getPendingSchedulingRequestsForAttempt(
attempt.getAppAttemptId());
if (schedulingRequestsRaw != null) {
for (SchedulingRequest req : schedulingRequestsRaw) {
resourceRequests.add(new ResourceRequestInfo(req));
}
}
}
}
}
// copy preemption info fields
RMAppMetrics appMetrics = app.getRMAppMetrics();
numAMContainerPreempted = appMetrics.getNumAMContainersPreempted();
preemptedResourceMB = appMetrics.getResourcePreempted().getMemorySize();
numNonAMContainerPreempted = appMetrics.getNumNonAMContainersPreempted();
preemptedResourceVCores =
appMetrics.getResourcePreempted().getVirtualCores();
memorySeconds = appMetrics.getMemorySeconds();
vcoreSeconds = appMetrics.getVcoreSeconds();
resourceSecondsMap = appMetrics.getResourceSecondsMap();
preemptedMemorySeconds = appMetrics.getPreemptedMemorySeconds();
preemptedVcoreSeconds = appMetrics.getPreemptedVcoreSeconds();
preemptedResourceSecondsMap = appMetrics.getPreemptedResourceSecondsMap();
ApplicationSubmissionContext appSubmissionContext =
app.getApplicationSubmissionContext();
unmanagedApplication = appSubmissionContext.getUnmanagedAM();
appNodeLabelExpression =
app.getApplicationSubmissionContext().getNodeLabelExpression();
/*
* When the deSelects parameter contains "amNodeLabelExpression", objects
* pertaining to the amNodeLabelExpression are not returned. By default,
* this is not skipped. (YARN-6871)
*/
if(!deSelects.contains(DeSelectType.AM_NODE_LABEL_EXPRESSION)) {
amNodeLabelExpression = (unmanagedApplication) ?
null :
app.getAMResourceRequests().get(0).getNodeLabelExpression();
}
/*
* When the deSelects parameter contains "appNodeLabelExpression", objects
* pertaining to the appNodeLabelExpression are not returned. By default,
* this is not skipped. (YARN-6871)
*/
if (!deSelects.contains(DeSelectType.APP_NODE_LABEL_EXPRESSION)) {
appNodeLabelExpression =
app.getApplicationSubmissionContext().getNodeLabelExpression();
}
/*
* When the deSelects parameter contains "amNodeLabelExpression", objects
* pertaining to the amNodeLabelExpression are not returned. By default,
* this is not skipped. (YARN-6871)
*/
if (!deSelects.contains(DeSelectType.AM_NODE_LABEL_EXPRESSION)) {
amNodeLabelExpression = (unmanagedApplication) ?
null :
app.getAMResourceRequests().get(0).getNodeLabelExpression();
}
/*
* When the deSelects parameter contains "resourceInfo", ResourceInfo
* objects are not returned. Default behavior is no skipping. (YARN-6871)
*/
// Setting partition based resource usage of application
if (!deSelects.contains(DeSelectType.RESOURCE_INFO)) {
ResourceScheduler scheduler = rm.getRMContext().getScheduler();
if (scheduler instanceof CapacityScheduler) {
RMAppAttempt attempt = app.getCurrentAppAttempt();
if (null != attempt) {
FiCaSchedulerApp ficaAppAttempt = ((CapacityScheduler) scheduler)
.getApplicationAttempt(attempt.getAppAttemptId());
resourceInfo = null != ficaAppAttempt ?
new ResourcesInfo(ficaAppAttempt.getSchedulingResourceUsage()) :
null;
}
}
}
/*
* When the deSelects parameter contains "appTimeouts", objects pertaining
* to app timeouts are not returned. By default, this is not skipped.
* (YARN-6871)
*/
if (!deSelects.contains(DeSelectType.TIMEOUTS)) {
Map<ApplicationTimeoutType, Long> applicationTimeouts =
app.getApplicationTimeouts();
timeouts = new AppTimeoutsInfo();
if (applicationTimeouts.isEmpty()) {
// If application is not set timeout, lifetime should be sent
// as default with expiryTime=UNLIMITED and remainingTime=-1
AppTimeoutInfo timeoutInfo = new AppTimeoutInfo();
timeoutInfo.setTimeoutType(ApplicationTimeoutType.LIFETIME);
timeouts.add(timeoutInfo);
} else {
for (Map.Entry<ApplicationTimeoutType, Long> entry : app
.getApplicationTimeouts().entrySet()) {
AppTimeoutInfo timeout = new AppTimeoutInfo();
timeout.setTimeoutType(entry.getKey());
long timeoutInMillis = entry.getValue();
timeout.setExpiryTime(Times.formatISO8601(timeoutInMillis));
if (app.isAppInCompletedStates()) {
timeout.setRemainingTime(0);
} else {
timeout.setRemainingTime(Math.max(
(timeoutInMillis - System.currentTimeMillis()) / 1000, 0));
}
timeouts.add(timeout);
}
}
}
}
}