|
@@ -18,8 +18,10 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.timelineservice;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.Map;
|
|
|
+import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
@@ -29,7 +31,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
-import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.ContainerEntity;
|
|
|
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity;
|
|
@@ -41,16 +42,15 @@ import org.apache.hadoop.yarn.client.api.TimelineClient;
|
|
|
import org.apache.hadoop.yarn.event.AsyncDispatcher;
|
|
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
+import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationContainerFinishedEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEvent;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEvent;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.LocalizationEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorImpl.ContainerMetric;
|
|
|
import org.apache.hadoop.yarn.util.ResourceCalculatorProcessTree;
|
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
@@ -72,9 +72,12 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
|
|
|
private String httpAddress;
|
|
|
|
|
|
+ protected final Map<ApplicationId, TimelineClient> appToClientMap;
|
|
|
+
|
|
|
public NMTimelinePublisher(Context context) {
|
|
|
super(NMTimelinePublisher.class.getName());
|
|
|
this.context = context;
|
|
|
+ appToClientMap = new ConcurrentHashMap<>();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -82,12 +85,6 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
dispatcher = new AsyncDispatcher();
|
|
|
dispatcher.register(NMTimelineEventType.class,
|
|
|
new ForwardingEventHandler());
|
|
|
- dispatcher
|
|
|
- .register(ContainerEventType.class, new ContainerEventHandler());
|
|
|
- dispatcher.register(ApplicationEventType.class,
|
|
|
- new ApplicationEventHandler());
|
|
|
- dispatcher.register(LocalizationEventType.class,
|
|
|
- new LocalizationEventDispatcher());
|
|
|
addIfService(dispatcher);
|
|
|
super.serviceInit(conf);
|
|
|
}
|
|
@@ -112,7 +109,6 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public void reportContainerResourceUsage(Container container, Long pmemUsage,
|
|
|
Float cpuUsagePercentPerCore) {
|
|
|
if (pmemUsage != ResourceCalculatorProcessTree.UNAVAILABLE ||
|
|
@@ -133,15 +129,32 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
Math.round(cpuUsagePercentPerCore));
|
|
|
entity.addMetric(cpuMetric);
|
|
|
}
|
|
|
- dispatcher.getEventHandler()
|
|
|
- .handle(new TimelinePublishEvent(entity, container.getContainerId()
|
|
|
- .getApplicationAttemptId().getApplicationId()));
|
|
|
+ ApplicationId appId = container.getContainerId().getApplicationAttemptId()
|
|
|
+ .getApplicationId();
|
|
|
+ try {
|
|
|
+ // no need to put it as part of publisher as timeline client already has
|
|
|
+ // Queuing concept
|
|
|
+ TimelineClient timelineClient = getTimelineClient(appId);
|
|
|
+ if (timelineClient != null) {
|
|
|
+ timelineClient.putEntitiesAsync(entity);
|
|
|
+ } else {
|
|
|
+ LOG.error("Seems like client has been removed before the container"
|
|
|
+ + " metric could be published for " + container.getContainerId());
|
|
|
+ }
|
|
|
+ } catch (IOException | YarnException e) {
|
|
|
+ LOG.error("Failed to publish Container metrics for container "
|
|
|
+ + container.getContainerId(), e);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void publishContainerCreatedEvent(ContainerEntity entity,
|
|
|
- ContainerId containerId, Resource resource, Priority priority,
|
|
|
- long timestamp) {
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
+ private void publishContainerCreatedEvent(ContainerEvent event) {
|
|
|
+ ContainerId containerId = event.getContainerID();
|
|
|
+ ContainerEntity entity = createContainerEntity(containerId);
|
|
|
+ Container container = context.getContainers().get(containerId);
|
|
|
+ Resource resource = container.getResource();
|
|
|
+
|
|
|
Map<String, Object> entityInfo = new HashMap<String, Object>();
|
|
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_MEMORY_ENTITY_INFO,
|
|
|
resource.getMemory());
|
|
@@ -152,7 +165,7 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_PORT_ENTITY_INFO,
|
|
|
nodeId.getPort());
|
|
|
entityInfo.put(ContainerMetricsConstants.ALLOCATED_PRIORITY_ENTITY_INFO,
|
|
|
- priority.toString());
|
|
|
+ container.getPriority().toString());
|
|
|
entityInfo.put(
|
|
|
ContainerMetricsConstants.ALLOCATED_HOST_HTTP_ADDRESS_ENTITY_INFO,
|
|
|
httpAddress);
|
|
@@ -160,13 +173,15 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
|
|
|
TimelineEvent tEvent = new TimelineEvent();
|
|
|
tEvent.setId(ContainerMetricsConstants.CREATED_EVENT_TYPE);
|
|
|
- tEvent.setTimestamp(timestamp);
|
|
|
+ tEvent.setTimestamp(event.getTimestamp());
|
|
|
|
|
|
entity.addEvent(tEvent);
|
|
|
- entity.setCreatedTime(timestamp);
|
|
|
- putEntity(entity, containerId.getApplicationAttemptId().getApplicationId());
|
|
|
+ entity.setCreatedTime(event.getTimestamp());
|
|
|
+ dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
|
|
|
+ containerId.getApplicationAttemptId().getApplicationId()));
|
|
|
}
|
|
|
|
|
|
+ @SuppressWarnings("unchecked")
|
|
|
private void publishContainerFinishedEvent(ContainerStatus containerStatus,
|
|
|
long timeStamp) {
|
|
|
ContainerId containerId = containerStatus.getContainerId();
|
|
@@ -186,7 +201,38 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
tEvent.setInfo(eventInfo);
|
|
|
|
|
|
entity.addEvent(tEvent);
|
|
|
- putEntity(entity, containerId.getApplicationAttemptId().getApplicationId());
|
|
|
+
|
|
|
+ dispatcher.getEventHandler().handle(new TimelinePublishEvent(entity,
|
|
|
+ containerId.getApplicationAttemptId().getApplicationId()));
|
|
|
+ }
|
|
|
+
|
|
|
+ private void publishContainerLocalizationEvent(
|
|
|
+ ContainerLocalizationEvent event, String eventType) {
|
|
|
+ Container container = event.getContainer();
|
|
|
+ ContainerId containerId = container.getContainerId();
|
|
|
+ TimelineEntity entity = createContainerEntity(containerId);
|
|
|
+
|
|
|
+ TimelineEvent tEvent = new TimelineEvent();
|
|
|
+ tEvent.setId(eventType);
|
|
|
+ tEvent.setTimestamp(event.getTimestamp());
|
|
|
+ entity.addEvent(tEvent);
|
|
|
+
|
|
|
+ ApplicationId appId =
|
|
|
+ container.getContainerId().getApplicationAttemptId().getApplicationId();
|
|
|
+ try {
|
|
|
+ // no need to put it as part of publisher as timeline client already has
|
|
|
+ // Queuing concept
|
|
|
+ TimelineClient timelineClient = getTimelineClient(appId);
|
|
|
+ if (timelineClient != null) {
|
|
|
+ timelineClient.putEntitiesAsync(entity);
|
|
|
+ } else {
|
|
|
+ LOG.error("Seems like client has been removed before the event could be"
|
|
|
+ + " published for " + container.getContainerId());
|
|
|
+ }
|
|
|
+ } catch (IOException | YarnException e) {
|
|
|
+ LOG.error("Failed to publish Container metrics for container "
|
|
|
+ + container.getContainerId(), e);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
private static ContainerEntity createContainerEntity(
|
|
@@ -207,23 +253,33 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
LOG.debug("Publishing the entity " + entity + ", JSON-style content: "
|
|
|
+ TimelineUtils.dumpTimelineRecordtoJSON(entity));
|
|
|
}
|
|
|
- TimelineClient timelineClient =
|
|
|
- context.getApplications().get(appId).getTimelineClient();
|
|
|
- timelineClient.putEntities(entity);
|
|
|
+ TimelineClient timelineClient = getTimelineClient(appId);
|
|
|
+ if (timelineClient != null) {
|
|
|
+ timelineClient.putEntities(entity);
|
|
|
+ } else {
|
|
|
+ LOG.error("Seems like client has been removed before the entity "
|
|
|
+ + "could be published for " + entity);
|
|
|
+ }
|
|
|
} catch (Exception e) {
|
|
|
LOG.error("Error when publishing entity " + entity, e);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public void publishApplicationEvent(ApplicationEvent event) {
|
|
|
// publish only when the desired event is received
|
|
|
switch (event.getType()) {
|
|
|
case INIT_APPLICATION:
|
|
|
case FINISH_APPLICATION:
|
|
|
- case APPLICATION_CONTAINER_FINISHED:
|
|
|
case APPLICATION_LOG_HANDLING_FAILED:
|
|
|
- dispatcher.getEventHandler().handle(event);
|
|
|
+ // TODO need to be handled in future,
|
|
|
+ // not sure to publish under which entity
|
|
|
+ break;
|
|
|
+ case APPLICATION_CONTAINER_FINISHED:
|
|
|
+ // this is actually used to publish the container Event
|
|
|
+ ApplicationContainerFinishedEvent evnt =
|
|
|
+ (ApplicationContainerFinishedEvent) event;
|
|
|
+ publishContainerFinishedEvent(evnt.getContainerStatus(),
|
|
|
+ event.getTimestamp());
|
|
|
break;
|
|
|
|
|
|
default:
|
|
@@ -235,12 +291,11 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public void publishContainerEvent(ContainerEvent event) {
|
|
|
// publish only when the desired event is received
|
|
|
switch (event.getType()) {
|
|
|
case INIT_CONTAINER:
|
|
|
- dispatcher.getEventHandler().handle(event);
|
|
|
+ publishContainerCreatedEvent(event);
|
|
|
break;
|
|
|
|
|
|
default:
|
|
@@ -253,15 +308,17 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("unchecked")
|
|
|
public void publishLocalizationEvent(LocalizationEvent event) {
|
|
|
// publish only when the desired event is received
|
|
|
switch (event.getType()) {
|
|
|
case CONTAINER_RESOURCES_LOCALIZED:
|
|
|
+ publishContainerLocalizationEvent((ContainerLocalizationEvent) event,
|
|
|
+ ContainerMetricsConstants.LOCALIZATION_FINISHED_EVENT_TYPE);
|
|
|
+ break;
|
|
|
case INIT_CONTAINER_RESOURCES:
|
|
|
- dispatcher.getEventHandler().handle(event);
|
|
|
+ publishContainerLocalizationEvent((ContainerLocalizationEvent) event,
|
|
|
+ ContainerMetricsConstants.LOCALIZATION_START_EVENT_TYPE);
|
|
|
break;
|
|
|
-
|
|
|
default:
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug(event.getType()
|
|
@@ -272,64 +329,6 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private class ApplicationEventHandler implements
|
|
|
- EventHandler<ApplicationEvent> {
|
|
|
- @Override
|
|
|
- public void handle(ApplicationEvent event) {
|
|
|
- switch (event.getType()) {
|
|
|
- case APPLICATION_CONTAINER_FINISHED:
|
|
|
- // this is actually used to publish the container Event
|
|
|
- ApplicationContainerFinishedEvent evnt =
|
|
|
- (ApplicationContainerFinishedEvent) event;
|
|
|
- publishContainerFinishedEvent(evnt.getContainerStatus(),
|
|
|
- event.getTimestamp());
|
|
|
- break;
|
|
|
- default:
|
|
|
- LOG.error("Seems like event type is captured only in "
|
|
|
- + "publishApplicationEvent method and not handled here");
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private class ContainerEventHandler implements EventHandler<ContainerEvent> {
|
|
|
- @Override
|
|
|
- public void handle(ContainerEvent event) {
|
|
|
- ContainerId containerId = event.getContainerID();
|
|
|
- Container container = context.getContainers().get(containerId);
|
|
|
- long timestamp = event.getTimestamp();
|
|
|
- ContainerEntity entity = createContainerEntity(containerId);
|
|
|
-
|
|
|
- switch (event.getType()) {
|
|
|
- case INIT_CONTAINER:
|
|
|
- publishContainerCreatedEvent(entity, containerId,
|
|
|
- container.getResource(), container.getPriority(), timestamp);
|
|
|
- break;
|
|
|
- default:
|
|
|
- LOG.error("Seems like event type is captured only in "
|
|
|
- + "publishContainerEvent method and not handled here");
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- private static final class LocalizationEventDispatcher implements
|
|
|
- EventHandler<LocalizationEvent> {
|
|
|
- @Override
|
|
|
- public void handle(LocalizationEvent event) {
|
|
|
- switch (event.getType()) {
|
|
|
- case INIT_CONTAINER_RESOURCES:
|
|
|
- case CONTAINER_RESOURCES_LOCALIZED:
|
|
|
- // TODO after priority based flush jira is finished
|
|
|
- break;
|
|
|
- default:
|
|
|
- LOG.error("Seems like event type is captured only in "
|
|
|
- + "publishLocalizationEvent method and not handled here");
|
|
|
- break;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* EventHandler implementation which forward events to NMMetricsPublisher.
|
|
|
* Making use of it, NMMetricsPublisher can avoid to have a public handle
|
|
@@ -363,4 +362,33 @@ public class NMTimelinePublisher extends CompositeService {
|
|
|
return entityToPublish;
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public void createTimelineClient(ApplicationId appId) {
|
|
|
+ if (!appToClientMap.containsKey(appId)) {
|
|
|
+ TimelineClient timelineClient =
|
|
|
+ TimelineClient.createTimelineClient(appId);
|
|
|
+ timelineClient.init(getConfig());
|
|
|
+ timelineClient.start();
|
|
|
+ appToClientMap.put(appId, timelineClient);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void stopTimelineClient(ApplicationId appId) {
|
|
|
+ TimelineClient client = appToClientMap.remove(appId);
|
|
|
+ if (client != null) {
|
|
|
+ client.stop();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void setTimelineServiceAddress(ApplicationId appId,
|
|
|
+ String collectorAddr) {
|
|
|
+ TimelineClient client = appToClientMap.get(appId);
|
|
|
+ if (client != null) {
|
|
|
+ client.setTimelineServiceAddress(collectorAddr);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private TimelineClient getTimelineClient(ApplicationId appId) {
|
|
|
+ return appToClientMap.get(appId);
|
|
|
+ }
|
|
|
}
|