|
@@ -18,11 +18,14 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.resourcemanager.metrics;
|
|
package org.apache.hadoop.yarn.server.resourcemanager.metrics;
|
|
|
|
|
|
|
|
+import java.util.ArrayList;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.Map;
|
|
import java.util.Map;
|
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
|
+import java.util.concurrent.Executors;
|
|
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
-import org.apache.commons.logging.Log;
|
|
|
|
-import org.apache.commons.logging.LogFactory;
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -32,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
|
|
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
|
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
|
|
|
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
|
|
import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants;
|
|
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
|
import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants;
|
|
@@ -44,6 +48,8 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
|
|
+import org.slf4j.Logger;
|
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
/**
|
|
/**
|
|
* This class is responsible for posting application, appattempt & Container
|
|
* This class is responsible for posting application, appattempt & Container
|
|
@@ -51,17 +57,63 @@ import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
|
*/
|
|
*/
|
|
public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
|
|
|
|
- private static final Log LOG =
|
|
|
|
- LogFactory.getLog(TimelineServiceV1Publisher.class);
|
|
|
|
|
|
+ private static final Logger LOG =
|
|
|
|
+ LoggerFactory.getLogger(TimelineServiceV1Publisher.class);
|
|
|
|
|
|
public TimelineServiceV1Publisher() {
|
|
public TimelineServiceV1Publisher() {
|
|
super("TimelineserviceV1Publisher");
|
|
super("TimelineserviceV1Publisher");
|
|
}
|
|
}
|
|
|
|
|
|
private TimelineClient client;
|
|
private TimelineClient client;
|
|
|
|
+ private LinkedBlockingQueue<TimelineEntity> entityQueue;
|
|
|
|
+ private ExecutorService sendEventThreadPool;
|
|
|
|
+ private int dispatcherPoolSize;
|
|
|
|
+ private int dispatcherBatchSize;
|
|
|
|
+ private int putEventInterval;
|
|
|
|
+ private boolean isTimeLineServerBatchEnabled;
|
|
|
|
+ private volatile boolean stopped = false;
|
|
|
|
+ private PutEventThread putEventThread;
|
|
|
|
+ private Object sendEntityLock;
|
|
|
|
|
|
@Override
|
|
@Override
|
|
protected void serviceInit(Configuration conf) throws Exception {
|
|
protected void serviceInit(Configuration conf) throws Exception {
|
|
|
|
+ isTimeLineServerBatchEnabled =
|
|
|
|
+ conf.getBoolean(
|
|
|
|
+ YarnConfiguration.RM_TIMELINE_SERVER_V1_PUBLISHER_BATCH_ENABLED,
|
|
|
|
+ YarnConfiguration.DEFAULT_RM_TIMELINE_SERVER_V1_PUBLISHER_BATCH_ENABLED);
|
|
|
|
+ if (isTimeLineServerBatchEnabled) {
|
|
|
|
+ putEventInterval =
|
|
|
|
+ conf.getInt(YarnConfiguration.RM_TIMELINE_SERVER_V1_PUBLISHER_INTERVAL,
|
|
|
|
+ YarnConfiguration.DEFAULT_RM_TIMELINE_SERVER_V1_PUBLISHER_INTERVAL)
|
|
|
|
+ * 1000;
|
|
|
|
+ if (putEventInterval <= 0) {
|
|
|
|
+ throw new IllegalArgumentException(
|
|
|
|
+ "RM_TIMELINE_SERVER_V1_PUBLISHER_INTERVAL should be greater than 0");
|
|
|
|
+ }
|
|
|
|
+ dispatcherPoolSize = conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE);
|
|
|
|
+ if (dispatcherPoolSize <= 0) {
|
|
|
|
+ throw new IllegalArgumentException(
|
|
|
|
+ "RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE should be greater than 0");
|
|
|
|
+ }
|
|
|
|
+ dispatcherBatchSize = conf.getInt(
|
|
|
|
+ YarnConfiguration.RM_TIMELINE_SERVER_V1_PUBLISHER_DISPATCHER_BATCH_SIZE,
|
|
|
|
+ YarnConfiguration.
|
|
|
|
+ DEFAULT_RM_TIMELINE_SERVER_V1_PUBLISHER_DISPATCHER_BATCH_SIZE);
|
|
|
|
+ if (dispatcherBatchSize <= 1) {
|
|
|
|
+ throw new IllegalArgumentException(
|
|
|
|
+ "RM_TIMELINE_SERVER_V1_PUBLISHER_DISPATCHER_BATCH_SIZE should be greater than 1");
|
|
|
|
+ }
|
|
|
|
+ putEventThread = new PutEventThread();
|
|
|
|
+ sendEventThreadPool = Executors.newFixedThreadPool(dispatcherPoolSize);
|
|
|
|
+ entityQueue = new LinkedBlockingQueue<>(dispatcherBatchSize + 1);
|
|
|
|
+ sendEntityLock = new Object();
|
|
|
|
+ LOG.info("Timeline service v1 batch publishing enabled");
|
|
|
|
+ } else {
|
|
|
|
+ LOG.info("Timeline service v1 batch publishing disabled");
|
|
|
|
+ }
|
|
client = TimelineClient.createTimelineClient();
|
|
client = TimelineClient.createTimelineClient();
|
|
addIfService(client);
|
|
addIfService(client);
|
|
super.serviceInit(conf);
|
|
super.serviceInit(conf);
|
|
@@ -69,6 +121,36 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
new TimelineV1EventHandler());
|
|
new TimelineV1EventHandler());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ protected void serviceStart() throws Exception {
|
|
|
|
+ if (isTimeLineServerBatchEnabled) {
|
|
|
|
+ stopped = false;
|
|
|
|
+ putEventThread.start();
|
|
|
|
+ }
|
|
|
|
+ super.serviceStart();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected void serviceStop() throws Exception {
|
|
|
|
+ super.serviceStop();
|
|
|
|
+ if (isTimeLineServerBatchEnabled) {
|
|
|
|
+ stopped = true;
|
|
|
|
+ putEventThread.interrupt();
|
|
|
|
+ try {
|
|
|
|
+ putEventThread.join();
|
|
|
|
+ SendEntity task = new SendEntity();
|
|
|
|
+ if (!task.buffer.isEmpty()) {
|
|
|
|
+ LOG.info("Initiating final putEntities, remaining entities left in entityQueue: {}",
|
|
|
|
+ task.buffer.size());
|
|
|
|
+ sendEventThreadPool.submit(task);
|
|
|
|
+ }
|
|
|
|
+ } finally {
|
|
|
|
+ sendEventThreadPool.shutdown();
|
|
|
|
+ if (!sendEventThreadPool.awaitTermination(3, TimeUnit.SECONDS)) {
|
|
|
|
+ sendEventThreadPool.shutdownNow();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public void appCreated(RMApp app, long createdTime) {
|
|
public void appCreated(RMApp app, long createdTime) {
|
|
@@ -244,7 +326,7 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
@SuppressWarnings("unchecked")
|
|
@SuppressWarnings("unchecked")
|
|
@Override
|
|
@Override
|
|
public void appAttemptFinished(RMAppAttempt appAttempt,
|
|
public void appAttemptFinished(RMAppAttempt appAttempt,
|
|
- RMAppAttemptState appAttemtpState, RMApp app, long finishedTime) {
|
|
|
|
|
|
+ RMAppAttemptState appAttemptState, RMApp app, long finishedTime) {
|
|
TimelineEntity entity =
|
|
TimelineEntity entity =
|
|
createAppAttemptEntity(appAttempt.getAppAttemptId());
|
|
createAppAttemptEntity(appAttempt.getAppAttemptId());
|
|
|
|
|
|
@@ -261,7 +343,7 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO,
|
|
eventInfo.put(AppAttemptMetricsConstants.FINAL_STATUS_INFO,
|
|
app.getFinalApplicationStatus().toString());
|
|
app.getFinalApplicationStatus().toString());
|
|
eventInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils
|
|
eventInfo.put(AppAttemptMetricsConstants.STATE_INFO, RMServerUtils
|
|
- .createApplicationAttemptState(appAttemtpState).toString());
|
|
|
|
|
|
+ .createApplicationAttemptState(appAttemptState).toString());
|
|
if (appAttempt.getMasterContainer() != null) {
|
|
if (appAttempt.getMasterContainer() != null) {
|
|
eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
|
|
eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_INFO,
|
|
appAttempt.getMasterContainer().getId().toString());
|
|
appAttempt.getMasterContainer().getId().toString());
|
|
@@ -361,23 +443,68 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
}
|
|
}
|
|
|
|
|
|
private void putEntity(TimelineEntity entity) {
|
|
private void putEntity(TimelineEntity entity) {
|
|
- try {
|
|
|
|
|
|
+ if (isTimeLineServerBatchEnabled) {
|
|
|
|
+ try {
|
|
|
|
+ entityQueue.put(entity);
|
|
|
|
+ if (entityQueue.size() > dispatcherBatchSize) {
|
|
|
|
+ SendEntity task = null;
|
|
|
|
+ synchronized (sendEntityLock) {
|
|
|
|
+ if (entityQueue.size() > dispatcherBatchSize) {
|
|
|
|
+ task = new SendEntity();
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ if (task != null) {
|
|
|
|
+ sendEventThreadPool.submit(task);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Error when publishing entity batch [ " + entity.getEntityType() + ","
|
|
|
|
+ + entity.getEntityId() + " ] ", e);
|
|
|
|
+ }
|
|
|
|
+ } else {
|
|
|
|
+ try {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Publishing the entity " + entity.getEntityId()
|
|
|
|
+ + ", JSON-style content: "
|
|
|
|
+ + TimelineUtils.dumpTimelineRecordtoJSON(entity));
|
|
|
|
+ }
|
|
|
|
+ client.putEntities(entity);
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Error when publishing entity [ " + entity.getEntityType() + ","
|
|
|
|
+ + entity.getEntityId() + " ] ", e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private class SendEntity implements Runnable {
|
|
|
|
+
|
|
|
|
+ private ArrayList<TimelineEntity> buffer;
|
|
|
|
+
|
|
|
|
+ SendEntity() {
|
|
|
|
+ buffer = new ArrayList();
|
|
|
|
+ entityQueue.drainTo(buffer);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
if (LOG.isDebugEnabled()) {
|
|
if (LOG.isDebugEnabled()) {
|
|
- LOG.debug("Publishing the entity " + entity.getEntityId()
|
|
|
|
- + ", JSON-style content: "
|
|
|
|
- + TimelineUtils.dumpTimelineRecordtoJSON(entity));
|
|
|
|
|
|
+ LOG.debug("Number of timeline entities being sent in batch: {}", buffer.size());
|
|
|
|
+ }
|
|
|
|
+ if (buffer.isEmpty()) {
|
|
|
|
+ return;
|
|
|
|
+ }
|
|
|
|
+ try {
|
|
|
|
+ client.putEntities(buffer.toArray(new TimelineEntity[0]));
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Error when publishing entity: ", e);
|
|
}
|
|
}
|
|
- client.putEntities(entity);
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- LOG.error("Error when publishing entity [" + entity.getEntityType() + ","
|
|
|
|
- + entity.getEntityId() + "]", e);
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
private class TimelineV1PublishEvent extends TimelinePublishEvent {
|
|
private class TimelineV1PublishEvent extends TimelinePublishEvent {
|
|
private TimelineEntity entity;
|
|
private TimelineEntity entity;
|
|
|
|
|
|
- public TimelineV1PublishEvent(SystemMetricsEventType type,
|
|
|
|
|
|
+ TimelineV1PublishEvent(SystemMetricsEventType type,
|
|
TimelineEntity entity, ApplicationId appId) {
|
|
TimelineEntity entity, ApplicationId appId) {
|
|
super(type, appId);
|
|
super(type, appId);
|
|
this.entity = entity;
|
|
this.entity = entity;
|
|
@@ -395,4 +522,46 @@ public class TimelineServiceV1Publisher extends AbstractSystemMetricsPublisher {
|
|
putEntity(event.getEntity());
|
|
putEntity(event.getEntity());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-}
|
|
|
|
|
|
+
|
|
|
|
+ private class PutEventThread extends Thread {
|
|
|
|
+ PutEventThread() {
|
|
|
|
+ super("PutEventThread");
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
|
|
+ LOG.info("System metrics publisher will put events every " +
|
|
|
|
+ String.valueOf(putEventInterval) + " milliseconds");
|
|
|
|
+ while (!stopped && !Thread.currentThread().isInterrupted()) {
|
|
|
|
+ if (System.currentTimeMillis() % putEventInterval >= 1000) {
|
|
|
|
+ try {
|
|
|
|
+ Thread.sleep(500);
|
|
|
|
+ } catch (InterruptedException e) {
|
|
|
|
+ LOG.warn(SystemMetricsPublisher.class.getName()
|
|
|
|
+ + " is interrupted. Exiting.");
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ continue;
|
|
|
|
+ }
|
|
|
|
+ SendEntity task = null;
|
|
|
|
+ synchronized (sendEntityLock) {
|
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
|
+ LOG.debug("Creating SendEntity task in PutEventThread");
|
|
|
|
+ }
|
|
|
|
+ task = new SendEntity();
|
|
|
|
+ }
|
|
|
|
+ if (task != null) {
|
|
|
|
+ sendEventThreadPool.submit(task);
|
|
|
|
+ }
|
|
|
|
+ try {
|
|
|
|
+ // sleep added to avoid multiple SendEntity task within a single interval.
|
|
|
|
+ Thread.sleep(1000);
|
|
|
|
+ } catch (InterruptedException e) {
|
|
|
|
+ LOG.warn(SystemMetricsPublisher.class.getName()
|
|
|
|
+ + " is interrupted. Exiting.");
|
|
|
|
+ break;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+}
|