|
@@ -30,6 +30,14 @@ import java.net.URL;
|
|
|
import java.net.URLConnection;
|
|
|
import java.security.GeneralSecurityException;
|
|
|
import java.security.PrivilegedExceptionAction;
|
|
|
+import java.util.concurrent.BlockingQueue;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.ExecutorService;
|
|
|
+import java.util.concurrent.Executors;
|
|
|
+import java.util.concurrent.FutureTask;
|
|
|
+import java.util.concurrent.LinkedBlockingQueue;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import javax.net.ssl.HostnameVerifier;
|
|
|
import javax.net.ssl.HttpsURLConnection;
|
|
@@ -130,6 +138,8 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
@VisibleForTesting
|
|
|
TimelineClientConnectionRetry connectionRetry;
|
|
|
|
|
|
+ private TimelineEntityDispatcher entityDispatcher;
|
|
|
+
|
|
|
// Abstract class for an operation that should be retried by timeline client
|
|
|
@Private
|
|
|
@VisibleForTesting
|
|
@@ -315,6 +325,7 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
serviceRetryInterval = conf.getLong(
|
|
|
YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS,
|
|
|
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS);
|
|
|
+ entityDispatcher = new TimelineEntityDispatcher(conf);
|
|
|
} else {
|
|
|
if (YarnConfiguration.useHttps(conf)) {
|
|
|
setTimelineServiceAddress(conf.get(
|
|
@@ -335,7 +346,9 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
|
|
|
@Override
|
|
|
protected void serviceStart() throws Exception {
|
|
|
- if (!timelineServiceV2) {
|
|
|
+ if (timelineServiceV2) {
|
|
|
+ entityDispatcher.start();
|
|
|
+ } else {
|
|
|
timelineWriter = createTimelineWriter(configuration, authUgi, client,
|
|
|
constructResURI(getConfig(), timelineServiceAddress, false));
|
|
|
}
|
|
@@ -357,6 +370,9 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
if (this.timelineWriter != null) {
|
|
|
this.timelineWriter.close();
|
|
|
}
|
|
|
+ if (timelineServiceV2) {
|
|
|
+ entityDispatcher.stop();
|
|
|
+ }
|
|
|
super.serviceStop();
|
|
|
}
|
|
|
|
|
@@ -376,37 +392,21 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
@Override
|
|
|
public void putEntities(
|
|
|
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
|
|
|
- throws IOException, YarnException {
|
|
|
- putEntities(false, entities);
|
|
|
+ throws IOException, YarnException {
|
|
|
+ if (!timelineServiceV2) {
|
|
|
+ throw new YarnException("v.2 method is invoked on a v.1.x client");
|
|
|
+ }
|
|
|
+ entityDispatcher.dispatchEntities(true, entities);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public void putEntitiesAsync(
|
|
|
org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
|
|
|
throws IOException, YarnException {
|
|
|
- putEntities(true, entities);
|
|
|
- }
|
|
|
-
|
|
|
- private void putEntities(boolean async,
|
|
|
- org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity... entities)
|
|
|
- throws IOException, YarnException {
|
|
|
if (!timelineServiceV2) {
|
|
|
throw new YarnException("v.2 method is invoked on a v.1.x client");
|
|
|
}
|
|
|
- org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities
|
|
|
- entitiesContainer =
|
|
|
- new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();
|
|
|
- for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity : entities) {
|
|
|
- entitiesContainer.addEntity(entity);
|
|
|
- }
|
|
|
- MultivaluedMap<String, String> params = new MultivaluedMapImpl();
|
|
|
- if (getContextAppId() != null) {
|
|
|
- params.add("appid", getContextAppId().toString());
|
|
|
- }
|
|
|
- if (async) {
|
|
|
- params.add("async", Boolean.TRUE.toString());
|
|
|
- }
|
|
|
- putObjects("entities", params, entitiesContainer);
|
|
|
+ entityDispatcher.dispatchEntities(false, entities);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -417,20 +417,10 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
|
|
|
// Used for new timeline service only
|
|
|
@Private
|
|
|
- public void putObjects(String path, MultivaluedMap<String, String> params,
|
|
|
+ protected void putObjects(String path, MultivaluedMap<String, String> params,
|
|
|
Object obj) throws IOException, YarnException {
|
|
|
|
|
|
- // timelineServiceAddress could haven't be initialized yet
|
|
|
- // or stale (only for new timeline service)
|
|
|
- int retries = pollTimelineServiceAddress(this.maxServiceRetries);
|
|
|
- if (timelineServiceAddress == null) {
|
|
|
- String errMessage = "TimelineClient has reached to max retry times : "
|
|
|
- + this.maxServiceRetries
|
|
|
- + ", but failed to fetch timeline service address. Please verify"
|
|
|
- + " Timeline Auxillary Service is configured in all the NMs";
|
|
|
- LOG.error(errMessage);
|
|
|
- throw new YarnException(errMessage);
|
|
|
- }
|
|
|
+ int retries = verifyRestEndPointAvailable();
|
|
|
|
|
|
// timelineServiceAddress could be stale, add retry logic here.
|
|
|
boolean needRetry = true;
|
|
@@ -448,6 +438,21 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private int verifyRestEndPointAvailable() throws YarnException {
|
|
|
+ // timelineServiceAddress could haven't be initialized yet
|
|
|
+ // or stale (only for new timeline service)
|
|
|
+ int retries = pollTimelineServiceAddress(this.maxServiceRetries);
|
|
|
+ if (timelineServiceAddress == null) {
|
|
|
+ String errMessage = "TimelineClient has reached to max retry times : "
|
|
|
+ + this.maxServiceRetries
|
|
|
+ + ", but failed to fetch timeline service address. Please verify"
|
|
|
+ + " Timeline Auxillary Service is configured in all the NMs";
|
|
|
+ LOG.error(errMessage);
|
|
|
+ throw new YarnException(errMessage);
|
|
|
+ }
|
|
|
+ return retries;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Check if reaching to maximum of retries.
|
|
|
* @param retries
|
|
@@ -641,7 +646,7 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
} catch (InterruptedException e) {
|
|
|
Thread.currentThread().interrupt();
|
|
|
}
|
|
|
- timelineServiceAddress = getTimelineServiceAddress();
|
|
|
+ // timelineServiceAddress = getTimelineServiceAddress();
|
|
|
retries--;
|
|
|
}
|
|
|
return retries;
|
|
@@ -906,4 +911,212 @@ public class TimelineClientImpl extends TimelineClient {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private final class EntitiesHolder extends FutureTask<Void> {
|
|
|
+ private final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities;
|
|
|
+ private final boolean isSync;
|
|
|
+
|
|
|
+ EntitiesHolder(
|
|
|
+ final org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities,
|
|
|
+ final boolean isSync) {
|
|
|
+ super(new Callable<Void>() {
|
|
|
+ // publishEntities()
|
|
|
+ public Void call() throws Exception {
|
|
|
+ MultivaluedMap<String, String> params = new MultivaluedMapImpl();
|
|
|
+ params.add("appid", contextAppId.toString());
|
|
|
+ params.add("async", Boolean.toString(!isSync));
|
|
|
+ putObjects("entities", params, entities);
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+ });
|
|
|
+ this.entities = entities;
|
|
|
+ this.isSync = isSync;
|
|
|
+ }
|
|
|
+
|
|
|
+ public boolean isSync() {
|
|
|
+ return isSync;
|
|
|
+ }
|
|
|
+
|
|
|
+ public org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities getEntities() {
|
|
|
+ return entities;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * This class is responsible for collecting the timeline entities and
|
|
|
+ * publishing them in async.
|
|
|
+ */
|
|
|
+ private class TimelineEntityDispatcher {
|
|
|
+ /**
|
|
|
+ * Time period for which the timelineclient will wait for draining after
|
|
|
+ * stop
|
|
|
+ */
|
|
|
+ private static final long DRAIN_TIME_PERIOD = 2000L;
|
|
|
+
|
|
|
+ private int numberOfAsyncsToMerge;
|
|
|
+ private final BlockingQueue<EntitiesHolder> timelineEntityQueue;
|
|
|
+ private ExecutorService executor;
|
|
|
+
|
|
|
+ TimelineEntityDispatcher(Configuration conf) {
|
|
|
+ timelineEntityQueue = new LinkedBlockingQueue<EntitiesHolder>();
|
|
|
+ numberOfAsyncsToMerge =
|
|
|
+ conf.getInt(YarnConfiguration.NUMBER_OF_ASYNC_ENTITIES_TO_MERGE,
|
|
|
+ YarnConfiguration.DEFAULT_NUMBER_OF_ASYNC_ENTITIES_TO_MERGE);
|
|
|
+ }
|
|
|
+
|
|
|
+ Runnable createRunnable() {
|
|
|
+ return new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ EntitiesHolder entitiesHolder;
|
|
|
+ while (!Thread.currentThread().isInterrupted()) {
|
|
|
+ // Merge all the async calls and make one push, but if its sync
|
|
|
+ // call push immediately
|
|
|
+ try {
|
|
|
+ entitiesHolder = timelineEntityQueue.take();
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ LOG.info("Timeline dispatcher thread was interrupted ");
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ if (entitiesHolder != null) {
|
|
|
+ publishWithoutBlockingOnQueue(entitiesHolder);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } finally {
|
|
|
+ if (!timelineEntityQueue.isEmpty()) {
|
|
|
+ LOG.info("Yet to publish " + timelineEntityQueue.size()
|
|
|
+ + " timelineEntities, draining them now. ");
|
|
|
+ }
|
|
|
+ // Try to drain the remaining entities to be published @ the max for
|
|
|
+ // 2 seconds
|
|
|
+ long timeTillweDrain =
|
|
|
+ System.currentTimeMillis() + DRAIN_TIME_PERIOD;
|
|
|
+ while (!timelineEntityQueue.isEmpty()) {
|
|
|
+ publishWithoutBlockingOnQueue(timelineEntityQueue.poll());
|
|
|
+ if (System.currentTimeMillis() > timeTillweDrain) {
|
|
|
+ // time elapsed stop publishing further....
|
|
|
+ if (!timelineEntityQueue.isEmpty()) {
|
|
|
+ LOG.warn("Time to drain elapsed! Remaining "
|
|
|
+ + timelineEntityQueue.size() + "timelineEntities will not"
|
|
|
+ + " be published");
|
|
|
+ // if some entities were not drained then we need interrupt
|
|
|
+ // the threads which had put sync EntityHolders to the queue.
|
|
|
+ EntitiesHolder nextEntityInTheQueue = null;
|
|
|
+ while ((nextEntityInTheQueue =
|
|
|
+ timelineEntityQueue.poll()) != null) {
|
|
|
+ nextEntityInTheQueue.cancel(true);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Publishes the given EntitiesHolder and return immediately if sync
|
|
|
+ * call, else tries to fetch the EntitiesHolder from the queue in non
|
|
|
+ * blocking fashion and collate the Entities if possible before
|
|
|
+ * publishing through REST.
|
|
|
+ *
|
|
|
+ * @param entitiesHolder
|
|
|
+ */
|
|
|
+ private void publishWithoutBlockingOnQueue(
|
|
|
+ EntitiesHolder entitiesHolder) {
|
|
|
+ if (entitiesHolder.isSync()) {
|
|
|
+ entitiesHolder.run();
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ int count = 1;
|
|
|
+ while (true) {
|
|
|
+ // loop till we find a sync put Entities or there is nothing
|
|
|
+ // to take
|
|
|
+ EntitiesHolder nextEntityInTheQueue = timelineEntityQueue.poll();
|
|
|
+ if (nextEntityInTheQueue == null) {
|
|
|
+ // Nothing in the queue just publish and get back to the
|
|
|
+ // blocked wait state
|
|
|
+ entitiesHolder.run();
|
|
|
+ break;
|
|
|
+ } else if (nextEntityInTheQueue.isSync()) {
|
|
|
+ // flush all the prev async entities first
|
|
|
+ entitiesHolder.run();
|
|
|
+ // and then flush the sync entity
|
|
|
+ nextEntityInTheQueue.run();
|
|
|
+ break;
|
|
|
+ } else {
|
|
|
+ // append all async entities together and then flush
|
|
|
+ entitiesHolder.getEntities().addEntities(
|
|
|
+ nextEntityInTheQueue.getEntities().getEntities());
|
|
|
+ count++;
|
|
|
+ if (count == numberOfAsyncsToMerge) {
|
|
|
+ // Flush the entities if the number of the async
|
|
|
+ // putEntites merged reaches the desired limit. To avoid
|
|
|
+ // collecting multiple entities and delaying for a long
|
|
|
+ // time.
|
|
|
+ entitiesHolder.run();
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
+
|
|
|
+ public void dispatchEntities(boolean sync,
|
|
|
+ org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity[] entitiesTobePublished)
|
|
|
+ throws YarnException {
|
|
|
+ if (executor.isShutdown()) {
|
|
|
+ throw new YarnException("Timeline client is in the process of stopping,"
|
|
|
+ + " not accepting any more TimelineEntities");
|
|
|
+ }
|
|
|
+
|
|
|
+ // wrap all TimelineEntity into TimelineEntities object
|
|
|
+ org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities entities =
|
|
|
+ new org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities();
|
|
|
+ for (org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity entity : entitiesTobePublished) {
|
|
|
+ entities.addEntity(entity);
|
|
|
+ }
|
|
|
+
|
|
|
+ // created a holder and place it in queue
|
|
|
+ EntitiesHolder entitiesHolder = new EntitiesHolder(entities, sync);
|
|
|
+ try {
|
|
|
+ timelineEntityQueue.put(entitiesHolder);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ throw new YarnException(
|
|
|
+ "Failed while adding entity to the queue for publishing", e);
|
|
|
+ }
|
|
|
+
|
|
|
+ if (sync) {
|
|
|
+ // In sync call we need to wait till its published and if any error then
|
|
|
+ // throw it back
|
|
|
+ try {
|
|
|
+ entitiesHolder.get();
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ throw new YarnException("Failed while publishing entity",
|
|
|
+ e.getCause());
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ throw new YarnException("Interrupted while publishing entity", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ public void start() {
|
|
|
+ executor = Executors.newSingleThreadExecutor();
|
|
|
+ executor.execute(createRunnable());
|
|
|
+ }
|
|
|
+
|
|
|
+ public void stop() {
|
|
|
+ LOG.info("Stopping TimelineClient.");
|
|
|
+ executor.shutdownNow();
|
|
|
+ try {
|
|
|
+ executor.awaitTermination(DRAIN_TIME_PERIOD, TimeUnit.MILLISECONDS);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ Thread.currentThread().interrupt();
|
|
|
+ e.printStackTrace();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|